From 0cc3ff782d1ff01701b6f45801c68a2923ca0231 Mon Sep 17 00:00:00 2001 From: Wish Date: Fri, 24 Nov 2023 10:07:37 +0800 Subject: [PATCH 01/25] Initial impl Signed-off-by: Wish --- cdc/sink/util/helper.go | 11 +- pkg/config/sink_protocol.go | 5 + pkg/errors/cdc_errors.go | 4 + pkg/sink/codec/builder/encoder_builder.go | 7 +- pkg/sink/codec/common/config.go | 6 + pkg/sink/codec/debezium/codec.go | 383 +++++++++++ pkg/sink/codec/debezium/encoder.go | 134 ++++ .../debezium/docker-compose.yml | 56 ++ tests/integration_tests/debezium/go.mod | 84 +++ tests/integration_tests/debezium/go.sum | 604 ++++++++++++++++++ .../debezium/sql/data_types.sql | 401 ++++++++++++ .../debezium/sql/debezium/LICENSE.txt | 202 ++++++ .../debezium/sql/debezium/README.txt | 3 + .../sql/debezium/binary_column_test.sql | 14 + .../sql/debezium/binary_mode_test.sql | 31 + .../connector_read_binary_field_test.sql | 18 + .../debezium/sql/debezium/connector_test.sql | 75 +++ .../sql/debezium/connector_test_ro.sql | 87 +++ .../sql/debezium/datetime_key_test.sql | 14 + .../sql/debezium/db_default_charset.sql | 5 + .../sql/debezium/db_default_charset_noutf.sql | 6 + .../sql/debezium/decimal_column_test.sql | 14 + .../debezium/default_value_all_zero_time.sql | 31 + .../sql/debezium/default_value_generated.sql | 7 + .../sql/debezium/enum_column_test.sql | 16 + .../debezium/sql/debezium/json_test.sqlx | 137 ++++ .../sql/debezium/multitable_dbz_871.sql | 10 + .../debezium/sql/debezium/mysql_dbz_6533.sql | 18 + .../debezium/nationalized_character_test.sql | 8 + .../sql/debezium/numeric_column_test.sql | 14 + .../debezium/sql/debezium/readbinlog_test.sql | 23 + .../debezium/sql/debezium/real_test.sql | 6 + .../debezium/sql/debezium/regression_test.sql | 180 ++++++ .../sql/debezium/skip_messages_test.sql | 6 + .../source_type_as_schema_parameter_test.sql | 16 + .../debezium/sql/debezium/strategy_test.sql | 39 ++ .../debezium/table_column_comment_test.sql | 12 + .../sql/debezium/timestamp_column_test.sql | 15 + .../debezium/sql/debezium/tinyint_test.sql | 23 + .../debezium/topic_name_sanitization_test.sql | 19 + .../sql/debezium/unsigned_integer_test.sql | 92 +++ .../debezium/sql/debezium/year_test.sql | 48 ++ tests/integration_tests/debezium/sql/dml.sql | 0 .../debezium/src/db_helper.go | 67 ++ .../integration_tests/debezium/src/logger.go | 12 + tests/integration_tests/debezium/src/main.go | 92 +++ .../debezium/src/test_cases.go | 219 +++++++ 47 files changed, 3271 insertions(+), 3 deletions(-) create mode 100644 pkg/sink/codec/debezium/codec.go create mode 100644 pkg/sink/codec/debezium/encoder.go create mode 100644 tests/integration_tests/debezium/docker-compose.yml create mode 100644 tests/integration_tests/debezium/go.mod create mode 100644 tests/integration_tests/debezium/go.sum create mode 100644 tests/integration_tests/debezium/sql/data_types.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/LICENSE.txt create mode 100644 tests/integration_tests/debezium/sql/debezium/README.txt create mode 100644 tests/integration_tests/debezium/sql/debezium/binary_column_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/connector_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/db_default_charset.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/db_default_charset_noutf.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/default_value_generated.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/enum_column_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/json_test.sqlx create mode 100644 tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/mysql_dbz_6533.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/real_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/regression_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/skip_messages_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/source_type_as_schema_parameter_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/strategy_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/tinyint_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql create mode 100644 tests/integration_tests/debezium/sql/debezium/year_test.sql create mode 100644 tests/integration_tests/debezium/sql/dml.sql create mode 100644 tests/integration_tests/debezium/src/db_helper.go create mode 100644 tests/integration_tests/debezium/src/logger.go create mode 100644 tests/integration_tests/debezium/src/main.go create mode 100644 tests/integration_tests/debezium/src/test_cases.go diff --git a/cdc/sink/util/helper.go b/cdc/sink/util/helper.go index cec952cbe5c..16cb77bbbd9 100644 --- a/cdc/sink/util/helper.go +++ b/cdc/sink/util/helper.go @@ -24,6 +24,7 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/sink/kafka" + "github.com/pingcap/tiflow/pkg/util" ) // GetTopic returns the topic name from the sink URI. @@ -79,7 +80,15 @@ func GetEncoderConfig( // Always set encoder's `MaxMessageBytes` equal to producer's `MaxMessageBytes` // to prevent that the encoder generate batched message too large // then cause producer meet `message too large`. - encoderConfig = encoderConfig.WithMaxMessageBytes(maxMsgBytes).WithChangefeedID(changefeedID) + encoderConfig = encoderConfig. + WithMaxMessageBytes(maxMsgBytes). + WithChangefeedID(changefeedID) + + tz, err := util.GetTimezone(config.GetGlobalServerConfig().TZ) + if err != nil { + return nil, cerror.WrapError(cerror.ErrSinkInvalidConfig, err) + } + encoderConfig.TimeZone = tz if err := encoderConfig.Validate(); err != nil { return nil, cerror.WrapError(cerror.ErrSinkInvalidConfig, err) diff --git a/pkg/config/sink_protocol.go b/pkg/config/sink_protocol.go index 0b4d4138059..da002e10146 100644 --- a/pkg/config/sink_protocol.go +++ b/pkg/config/sink_protocol.go @@ -38,6 +38,7 @@ const ( ProtocolCraft ProtocolOpen ProtocolCsv + ProtocolDebezium ) // IsBatchEncode returns whether the protocol is a batch encoder. @@ -66,6 +67,8 @@ func ParseSinkProtocolFromString(protocol string) (Protocol, error) { return ProtocolOpen, nil case "csv": return ProtocolCsv, nil + case "debezium": + return ProtocolDebezium, nil default: return ProtocolUnknown, cerror.ErrSinkUnknownProtocol.GenWithStackByArgs(protocol) } @@ -90,6 +93,8 @@ func (p Protocol) String() string { return "open-protocol" case ProtocolCsv: return "csv" + case ProtocolDebezium: + return "debezium" default: panic("unreachable") } diff --git a/pkg/errors/cdc_errors.go b/pkg/errors/cdc_errors.go index 22b60205b79..b366753b1b5 100644 --- a/pkg/errors/cdc_errors.go +++ b/pkg/errors/cdc_errors.go @@ -411,6 +411,10 @@ var ( "csv decode failed", errors.RFCCodeText("CDC:ErrCSVDecodeFailed"), ) + ErrDebeziumEncodeFailed = errors.Normalize( + "debezium encode failed", + errors.RFCCodeText("CDC:ErrDebeziumEncodeFailed"), + ) ErrStorageSinkInvalidConfig = errors.Normalize( "storage sink config invalid", errors.RFCCodeText("CDC:ErrStorageSinkInvalidConfig"), diff --git a/pkg/sink/codec/builder/encoder_builder.go b/pkg/sink/codec/builder/encoder_builder.go index 49e58543abc..ad1f096ab6c 100644 --- a/pkg/sink/codec/builder/encoder_builder.go +++ b/pkg/sink/codec/builder/encoder_builder.go @@ -24,13 +24,15 @@ import ( "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/sink/codec/craft" "github.com/pingcap/tiflow/pkg/sink/codec/csv" + "github.com/pingcap/tiflow/pkg/sink/codec/debezium" "github.com/pingcap/tiflow/pkg/sink/codec/maxwell" "github.com/pingcap/tiflow/pkg/sink/codec/open" ) // NewRowEventEncoderBuilder returns an RowEventEncoderBuilder func NewRowEventEncoderBuilder( - ctx context.Context, cfg *common.Config, + ctx context.Context, + cfg *common.Config, ) (codec.RowEventEncoderBuilder, error) { switch cfg.Protocol { case config.ProtocolDefault, config.ProtocolOpen: @@ -45,7 +47,8 @@ func NewRowEventEncoderBuilder( return canal.NewJSONRowEventEncoderBuilder(ctx, cfg) case config.ProtocolCraft: return craft.NewBatchEncoderBuilder(cfg), nil - + case config.ProtocolDebezium: + return debezium.NewBatchEncoderBuilder(cfg), nil default: return nil, cerror.ErrSinkUnknownProtocol.GenWithStackByArgs(cfg.Protocol) } diff --git a/pkg/sink/codec/common/config.go b/pkg/sink/codec/common/config.go index 8e7ba8f241a..ce91c68d975 100644 --- a/pkg/sink/codec/common/config.go +++ b/pkg/sink/codec/common/config.go @@ -16,6 +16,7 @@ package common import ( "net/http" "net/url" + "time" "github.com/gin-gonic/gin/binding" "github.com/imdario/mergo" @@ -70,6 +71,9 @@ type Config struct { // for open protocol OnlyOutputUpdatedColumns bool + + // Currently only Debezium protocol is aware of the time zone + TimeZone *time.Location } // NewConfig return a Config for codec @@ -91,6 +95,8 @@ func NewConfig(protocol config.Protocol) *Config { OnlyOutputUpdatedColumns: false, DeleteOnlyHandleKeyColumns: false, LargeMessageHandle: config.NewDefaultLargeMessageHandleConfig(), + + TimeZone: time.Local, } } diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go new file mode 100644 index 00000000000..39fd68952ab --- /dev/null +++ b/pkg/sink/codec/debezium/codec.go @@ -0,0 +1,383 @@ +package debezium + +import ( + "encoding/base64" + "encoding/binary" + "encoding/json" + "fmt" + "strconv" + "strings" + "time" + + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/tikv/pd/pkg/utils/tsoutil" +) + +type debeziumDataChangeMsg struct { + // schema is unsupported + Payload *debeziumDataChangeMsgPayload `json:"payload"` +} + +type debeziumDataChangeMsgPayload struct { + // Before: An optional field that specifies the state of the row before the event occurred. + // When the op field is c for create, the before field is null since this change event is for new content. + // In a delete event value, the before field contains the values that were in the row before + // it was deleted with the database commit. + Before map[string]any `json:"before"` + // After: An optional field that specifies the state of the row after the event occurred. + // Optional field that specifies the state of the row after the event occurred. + // In a delete event value, the after field is null, signifying that the row no longer exists. + After map[string]any `json:"after"` + Source *debeziumMsgSource `json:"source"` + // Op: Mandatory string that describes the type of operation that caused the connector to generate the event. + // Valid values are: + // c = create + // u = update + // d = delete + // r = read (applies to only snapshots) + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events + Op string `json:"op"` + // TsMs: displays the time at which the connector processed the event + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events + TsMs int64 `json:"ts_ms"` + // Transaction: Always null + Transaction *struct{} `json:"transaction"` +} + +type debeziumMsgSource struct { + Version string `json:"version"` + Connector string `json:"connector"` + Name string `json:"name"` + // TsMs: In the source object, ts_ms indicates the time that the change was made in the database. + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events + TsMs int64 `json:"ts_ms"` + Snapshot bool `json:"snapshot"` + Db string `json:"db"` + Table string `json:"table"` + ServerID int64 `json:"server_id"` + GtID *string `json:"gtid"` + File string `json:"file"` + Pos int64 `json:"pos"` + Row int32 `json:"row"` + Thread int64 `json:"thread"` + Query *string `json:"query"` + + // The followings are TiDB extended fields + CommitTs uint64 `json:"commit_ts"` + ClusterID string `json:"cluster_id"` +} + +type Codec struct { + config *common.Config +} + +// See https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-data-types +func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) (any, error) { + if col.Value == nil { + return nil, nil + } + switch col.Type { + case mysql.TypeBit: + if v, ok := col.Value.(uint64); ok { + // Debezium behavior: + // BIT(1) → BOOLEAN + // BIT(>1) → BYTES The byte[] contains the bits in little-endian form and is sized to + // contain the specified number of bits. + n := ft.GetFlen() + if n == 1 { + return v != 0, nil + } else { + buf := make([]byte, 8) + binary.LittleEndian.PutUint64(buf, uint64(v)) + + numBytes := n / 8 + if n%8 != 0 { + numBytes += 1 + } + return base64.StdEncoding.EncodeToString(buf[:numBytes]), nil + } + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for bit column %s", + col.Value, + col.Name)) + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, + mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + if col.Flag.IsBinary() { + if v, ok := col.Value.([]byte); ok { + return base64.StdEncoding.EncodeToString(v), nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for binary string column %s", + col.Value, + col.Name)) + } else { + if v, ok := col.Value.([]byte); ok { + return string(v), nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for non-binary string column %s", + col.Value, + col.Name)) + } + case mysql.TypeEnum: + if v, ok := col.Value.(uint64); ok { + enumVar, err := types.ParseEnumValue(ft.GetElems(), v) + if err != nil { + return nil, cerror.WrapError(cerror.ErrDebeziumEncodeFailed, err) + } + return enumVar.Name, nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for enum column %s", + col.Value, + col.Name)) + case mysql.TypeSet: + if v, ok := col.Value.(uint64); ok { + setVar, err := types.ParseSetValue(ft.GetElems(), v) + if err != nil { + return nil, cerror.WrapError(cerror.ErrDebeziumEncodeFailed, err) + } + return setVar.Name, nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for set column %s", + col.Value, + col.Name)) + case mysql.TypeNewDecimal: + if v, ok := col.Value.(string); ok { + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + err) + } + return floatV, nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for decimal column %s", + col.Value, + col.Name)) + case mysql.TypeDate, mysql.TypeNewDate: + if v, ok := col.Value.(string); ok { + if v == "0000-00-00" { + if mysql.HasNotNullFlag(ft.GetFlag()) { + return 0, nil + } else { + return nil, nil + } + } + t, err := time.Parse("2006-01-02", v) + if err != nil { + // For example, time may be invalid like 1000-00-00 + return nil, nil + } + return t.Unix() / 60 / 60 / 24, nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for date column %s", + col.Value, + col.Name)) + case mysql.TypeDatetime: + // Debezium behavior from doc: + // > Such columns are converted into epoch milliseconds or microseconds based on the + // > column's precision by using UTC. + + // TODO: For Default Value = CURRENT_TIMESTAMP, the result is incorrect. + if v, ok := col.Value.(string); ok { + if strings.HasPrefix(v, "0000-00-00") { + if mysql.HasNotNullFlag(ft.GetFlag()) { + return 0, nil + } else { + return nil, nil + } + } + + t, err := time.Parse("2006-01-02 15:04:05.999999", v) + if err != nil { + // For example, time may be 1000-00-00 + return nil, nil + } + if ft.GetDecimal() <= 3 { + return t.UnixMilli(), nil + } else { + return t.UnixMicro(), nil + } + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for datetime column %s", + col.Value, + col.Name)) + case mysql.TypeTimestamp: + // Debezium behavior from doc: + // > The TIMESTAMP type represents a timestamp without time zone information. + // > It is converted by MySQL from the server (or session's) current time zone into UTC + // > when writing and from UTC into the server (or session's) current time zone when reading + // > back the value. + // > Such columns are converted into an equivalent io.debezium.time.ZonedTimestamp in UTC + // > based on the server (or session's) current time zone. The time zone will be queried from + // > the server by default. If this fails, it must be specified explicitly by the database + // > connectionTimeZone MySQL configuration option. + if v, ok := col.Value.(string); ok { + // In cdc/entry/codec.go/unflatten, TIMESTAMP types are formatted + // in a timezone comes from config.GetGlobalServerConfig().TZ + // So here we must parse it in the same time zone, + // and then output in UTC as Debezium requires. + t, err := time.ParseInLocation("2006-01-02 15:04:05.999999", v, c.config.TimeZone) + if err != nil { + // For example, time may be invalid like 1000-00-00 + return nil, nil + } + + str := t.UTC().Format("2006-01-02T15:04:05") + fsp := ft.GetDecimal() + if fsp > 0 { + tmp := fmt.Sprintf(".%06d", t.Nanosecond()/1000) + str = str + tmp[:1+fsp] + } + str += "Z" + + return str, nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for timestamp column %s", + col.Value, + col.Name)) + case mysql.TypeDuration: + // Debezium behavior from doc: + // > Represents the time value in microseconds and does not include + // > time zone information. MySQL allows M to be in the range of 0-6. + if v, ok := col.Value.(string); ok { + ctx := &stmtctx.StatementContext{} + d, _, _, err := types.StrToDuration(ctx, v, ft.GetDecimal()) + if err != nil { + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + err) + } + return d.Microseconds(), nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for time column %s", + col.Value, + col.Name)) + default: + return col.Value, nil + } +} + +func (c *Codec) rowChangeToDebeziumMsg(e *model.RowChangedEvent) (*debeziumDataChangeMsg, error) { + commitTime, _ := tsoutil.ParseTS(e.CommitTs) + + source := &debeziumMsgSource{ + Version: "2.4.0.Final", + Connector: "TiCDC", + Name: "", // TODO + TsMs: commitTime.UnixMilli(), + Snapshot: false, + Db: e.Table.Schema, + Table: e.Table.Table, + ServerID: 0, + GtID: nil, + File: "", + Pos: 0, + Row: 0, + Thread: 0, + Query: nil, + + CommitTs: e.CommitTs, + ClusterID: "", // TODO + } + + payload := &debeziumDataChangeMsgPayload{ + Source: source, + TsMs: time.Now().UnixMilli(), + Transaction: nil, + } + + applyBefore := func() error { + payload.Before = make(map[string]any) + for i, col := range e.PreColumns { + value, err := c.convertToDebeziumField(col, e.ColInfos[i].Ft) + if err != nil { + return err + } + payload.Before[col.Name] = value + } + return nil + } + + applyAfter := func() error { + payload.After = make(map[string]any) + for i, col := range e.Columns { + value, err := c.convertToDebeziumField(col, e.ColInfos[i].Ft) + if err != nil { + return err + } + payload.After[col.Name] = value + } + return nil + } + + if e.IsInsert() { + payload.Op = "c" + payload.Before = nil + if err := applyAfter(); err != nil { + return nil, err + } + } else if e.IsDelete() { + payload.Op = "d" + payload.After = nil + if err := applyBefore(); err != nil { + return nil, err + } + } else if e.IsUpdate() { + payload.Op = "u" + if err := applyBefore(); err != nil { + return nil, err + } + if err := applyAfter(); err != nil { + return nil, err + } + } + + return &debeziumDataChangeMsg{ + Payload: payload, + }, nil +} + +func (c *Codec) EncodeRowChangedEvent( + e *model.RowChangedEvent, +) ([]byte, error) { + m, err := c.rowChangeToDebeziumMsg(e) + if err != nil { + return nil, errors.Trace(err) + } + return json.Marshal(m) +} diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go new file mode 100644 index 00000000000..3d4a45edd8a --- /dev/null +++ b/pkg/sink/codec/debezium/encoder.go @@ -0,0 +1,134 @@ +// Copyright 2023 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 debezium + +import ( + "bytes" + "context" + "encoding/binary" + + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/sink/codec" + "github.com/pingcap/tiflow/pkg/sink/codec/common" +) + +type BatchEncoder struct { + keyBuf *bytes.Buffer + valueBuf *bytes.Buffer + callbackBuf []func() + batchSize int + + config *common.Config + codec *Codec +} + +// EncodeCheckpointEvent implements the RowEventEncoder interface +func (d *BatchEncoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { + // Currently ignored. Debezium MySQL Connector does not emit such event. + return nil, nil +} + +// AppendRowChangedEvent implements the RowEventEncoder interface +func (d *BatchEncoder) AppendRowChangedEvent( + _ context.Context, + _ string, + e *model.RowChangedEvent, + callback func(), +) error { + value, err := d.codec.EncodeRowChangedEvent(e) + if err != nil { + return errors.Trace(err) + } + d.valueBuf.Write(value) + d.batchSize++ + if callback != nil { + d.callbackBuf = append(d.callbackBuf, callback) + } + return nil +} + +// EncodeDDLEvent implements the RowEventEncoder interface +// DDL message unresolved tso +func (d *BatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message, error) { + // Schema Change Events are currently not supported. + return nil, nil +} + +// Build implements the RowEventEncoder interface +func (d *BatchEncoder) Build() []*common.Message { + if d.batchSize == 0 { + return nil + } + + ret := common.NewMsg(config.ProtocolDebezium, + d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MessageTypeRow, nil, nil) + ret.SetRowsCount(d.batchSize) + if len(d.callbackBuf) != 0 && len(d.callbackBuf) == d.batchSize { + callbacks := d.callbackBuf + ret.Callback = func() { + for _, cb := range callbacks { + cb() + } + } + d.callbackBuf = make([]func(), 0) + } + d.reset() + return []*common.Message{ret} +} + +// reset implements the RowEventEncoder interface +func (d *BatchEncoder) reset() { + d.keyBuf.Reset() + d.valueBuf.Reset() + d.batchSize = 0 + var versionByte [8]byte + binary.BigEndian.PutUint64(versionByte[:], codec.BatchVersion1) + d.keyBuf.Write(versionByte[:]) +} + +// newBatchEncoder creates a new Debezium BatchEncoder. +func newBatchEncoder(config *common.Config) codec.RowEventEncoder { + batch := &BatchEncoder{ + keyBuf: &bytes.Buffer{}, + valueBuf: &bytes.Buffer{}, + callbackBuf: make([]func(), 0), + config: config, + codec: &Codec{ + config: config, + }, + } + batch.reset() + return batch +} + +type batchEncoderBuilder struct { + config *common.Config +} + +// NewBatchEncoderBuilder creates a Debezium batchEncoderBuilder. +func NewBatchEncoderBuilder(config *common.Config) codec.RowEventEncoderBuilder { + return &batchEncoderBuilder{ + config: config, + } +} + +// Build a `maxwellBatchEncoder` +func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { + return newBatchEncoder(b.config) +} + +// CleanMetrics do nothing +func (b *batchEncoderBuilder) CleanMetrics() {} diff --git a/tests/integration_tests/debezium/docker-compose.yml b/tests/integration_tests/debezium/docker-compose.yml new file mode 100644 index 00000000000..b6cf14c9153 --- /dev/null +++ b/tests/integration_tests/debezium/docker-compose.yml @@ -0,0 +1,56 @@ +version: "2" +services: + zookeeper: + restart: always + image: quay.io/debezium/zookeeper:2.4 + ports: + - 2181:2181 + - 2888:2888 + - 3888:3888 + kafka: + restart: always + image: quay.io/debezium/kafka:2.4 + ports: + - 9092:9092 + - 9094:9094 + depends_on: + - zookeeper + environment: + - ZOOKEEPER_CONNECT=zookeeper:2181 + - KAFKA_LISTENERS=INTERNAL://0.0.0.0:9092,OUTSIDE://0.0.0.0:9094 + - KAFKA_ADVERTISED_LISTENERS=INTERNAL://kafka:9092,OUTSIDE://localhost:9094 + - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=INTERNAL:PLAINTEXT,OUTSIDE:PLAINTEXT + - KAFKA_INTER_BROKER_LISTENER_NAME=INTERNAL + mysql: + restart: always + ports: + - 3306:3306 + image: quay.io/debezium/example-mysql:2.4 + environment: + - MYSQL_ROOT_PASSWORD= + - MYSQL_ALLOW_EMPTY_PASSWORD=yes + - MYSQL_USER=mysqluser + - MYSQL_PASSWORD=mysqlpw + connect: + restart: always + image: quay.io/debezium/connect:2.4 + ports: + - 8083:8083 + depends_on: + - kafka + - mysql + environment: + - BOOTSTRAP_SERVERS=kafka:9092 + - GROUP_ID=1 + - CONFIG_STORAGE_TOPIC=my_connect_configs + - OFFSET_STORAGE_TOPIC=my_connect_offsets + - STATUS_STORAGE_TOPIC=my_connect_statuses + # watcher: # For Debug Purpose + # restart: always + # image: quay.io/debezium/kafka:2.4 + # depends_on: + # - kafka + # command: watch-topic -a -k output_debezium + # environment: + # - ZOOKEEPER_CONNECT=zookeeper:2181 + # - KAFKA_BROKER=kafka:9092 diff --git a/tests/integration_tests/debezium/go.mod b/tests/integration_tests/debezium/go.mod new file mode 100644 index 00000000000..866df734a63 --- /dev/null +++ b/tests/integration_tests/debezium/go.mod @@ -0,0 +1,84 @@ +module github.com/breezewish/checker + +go 1.21.0 + +require ( + github.com/go-sql-driver/mysql v1.7.1 + github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873 + github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4 + github.com/thessem/zap-prettyconsole v0.3.0 + go.uber.org/zap v1.26.0 +) + +require ( + github.com/Code-Hex/dd v1.1.0 // indirect + github.com/alecthomas/chroma v0.10.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/cockroachdb/errors v1.8.1 // indirect + github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f // indirect + github.com/cockroachdb/redact v1.0.8 // indirect + github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 // indirect + github.com/coreos/go-semver v0.3.1 // indirect + github.com/coreos/go-systemd/v22 v22.5.0 // indirect + github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect + github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 // indirect + github.com/dlclark/regexp2 v1.4.0 // indirect + github.com/fatih/color v1.16.0 // indirect + github.com/go-ole/go-ole v1.2.6 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/go-cmp v0.6.0 // indirect + github.com/google/uuid v1.3.1 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect + github.com/klauspost/compress v1.17.1 // indirect + github.com/kr/pretty v0.3.1 // indirect + github.com/kr/text v0.2.0 // indirect + github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect + github.com/mattn/go-colorable v0.1.13 // indirect + github.com/mattn/go-isatty v0.0.20 // indirect + github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pierrec/lz4/v4 v4.1.15 // indirect + github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 // indirect + github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect + github.com/pingcap/kvproto v0.0.0-20230925123611-87bebcc0d071 // indirect + github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 // indirect + github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect + github.com/prometheus/client_golang v1.17.0 // indirect + github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/common v0.45.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect + github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect + github.com/rogpeppe/go-internal v1.11.0 // indirect + github.com/segmentio/kafka-go v0.4.45 // indirect + github.com/shirou/gopsutil/v3 v3.23.10 // indirect + github.com/shoenig/go-m1cpu v0.1.6 // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e // indirect + github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865 // indirect + github.com/tklauser/go-sysconf v0.3.12 // indirect + github.com/tklauser/numcpus v0.6.1 // indirect + github.com/twmb/murmur3 v1.1.6 // indirect + github.com/yusufpapurcu/wmi v1.2.3 // indirect + go.etcd.io/etcd/api/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect + go.etcd.io/etcd/client/v3 v3.5.10 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + golang.org/x/exp v0.0.0-20231006140011-7918f672742d // indirect + golang.org/x/net v0.18.0 // indirect + golang.org/x/sync v0.4.0 // indirect + golang.org/x/sys v0.14.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect + google.golang.org/grpc v1.59.0 // indirect + google.golang.org/protobuf v1.31.0 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect +) diff --git a/tests/integration_tests/debezium/go.sum b/tests/integration_tests/debezium/go.sum new file mode 100644 index 00000000000..3fe1773d286 --- /dev/null +++ b/tests/integration_tests/debezium/go.sum @@ -0,0 +1,604 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= +github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 h1:mFRzDkZVAjdal+s7s0MwaRv9igoPqLRdzOLzw/8Xvq8= +github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358/go.mod h1:chxPXzSsl7ZWRAuOIE23GDNzjWuZquvFlgA8xmpunjU= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/toml v1.3.2 h1:o7IhLm0Msx3BaB+n3Ag7L8EVlByGnpq14C4YWiu/gL8= +github.com/BurntSushi/toml v1.3.2/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= +github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= +github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= +github.com/Code-Hex/dd v1.1.0 h1:VEtTThnS9l7WhpKUIpdcWaf0B8Vp0LeeSEsxA1DZseI= +github.com/Code-Hex/dd v1.1.0/go.mod h1:VaMyo/YjTJ3d4qm/bgtrUkT2w+aYwJ07Y7eCWyrJr1w= +github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= +github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= +github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= +github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= +github.com/alecthomas/chroma v0.10.0 h1:7XDcGkCQopCNKjZHfYrNLraA+M7e0fMiJ/Mfikbfjek= +github.com/alecthomas/chroma v0.10.0/go.mod h1:jtJATyUxlIORhUOFNA9NZDWGAQ8wpxQQqNSB4rjA/1s= +github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= +github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= +github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= +github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= +github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= +github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= +github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= +github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= +github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= +github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= +github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= +github.com/coocood/freecache v1.2.1 h1:/v1CqMq45NFH9mp/Pt142reundeBM0dVUD3osQBeu/U= +github.com/coocood/freecache v1.2.1/go.mod h1:RBUWa/Cy+OHdfTGFEhEuE1pMCMX51Ncizj7rthiQ3vk= +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/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-semver v0.3.1 h1:yi21YpKnrx1gt5R+la8n5WgS0kCrsPp33dmEyHReZr4= +github.com/coreos/go-semver v0.3.1/go.mod h1:irMmmIw/7yzSRPWryHsK7EYSg09caPQL03VsM8rvUec= +github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= +github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= +github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= +github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= +github.com/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= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= +github.com/dgraph-io/ristretto v0.1.1 h1:6CWw5tJNgpegArSHpNHJKldNeq03FQCwYvfMVWajOK8= +github.com/dgraph-io/ristretto v0.1.1/go.mod h1:S1GPSBCYCIhmVNfcth17y2zZtQT6wzkzgwUve0VDWWA= +github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= +github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= +github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dlclark/regexp2 v1.4.0 h1:F1rxgk7p4uKjwIQxBs9oAXe5CqrXlCduYEJvrF4u93E= +github.com/dlclark/regexp2 v1.4.0/go.mod h1:2pZnwuY/m+8K6iRw6wQdMtk+rH5tNGR1i55kozfMjCc= +github.com/dolthub/maphash v0.1.0 h1:bsQ7JsF4FkkWyrP3oCnFJgrCUAFbFf3kOl4L/QxPDyQ= +github.com/dolthub/maphash v0.1.0/go.mod h1:gkg4Ch4CdCDu5h6PMriVLawB7koZ+5ijb9puGMV50a4= +github.com/dolthub/swiss v0.2.1 h1:gs2osYs5SJkAaH5/ggVJqXQxRXtWshF6uE0lgR/Y3Gw= +github.com/dolthub/swiss v0.2.1/go.mod h1:8AhKZZ1HK7g18j7v7k6c5cYIGEZJcPn0ARsai8cUrh0= +github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= +github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= +github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= +github.com/fatih/color v1.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM= +github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE= +github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= +github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= +github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= +github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= +github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= +github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= +github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= +github.com/go-asn1-ber/asn1-ber v1.5.4 h1:vXT6d/FNDiELJnLb6hGNa309LMsrCoYFvpwHDF0+Y1A= +github.com/go-asn1-ber/asn1-ber v1.5.4/go.mod h1:hEBeB/ic+5LoWskz+yKT7vGhhPYkProFKoKdwZRWMe0= +github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= +github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= +github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-ldap/ldap/v3 v3.4.4 h1:qPjipEpt+qDa6SI/h1fzuGWoRUY+qqQ9sOZq67/PYUs= +github.com/go-ldap/ldap/v3 v3.4.4/go.mod h1:fe1MsuN5eJJ1FeLT/LEBVdWfNWKh459R7aXgXtJC+aI= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= +github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= +github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrtU8EI= +github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= +github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= +github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= +github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= +github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= +github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/glog v1.1.2 h1:DVjP2PbBOzHyzA+dn3WhHIq4NdVu3Q+pvivFICf/7fo= +github.com/golang/glog v1.1.2/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= +github.com/google/btree v1.1.2 h1:xf4v41cLI2Z6FxbKm+8Bu+m8ifhj15JuZ9sa0jZCMUU= +github.com/google/btree v1.1.2/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= +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/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= +github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= +github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= +github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= +github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= +github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= +github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/influxdata/tdigest v0.0.1 h1:XpFptwYmnEKUqmkcDjrzffswZ3nvNeevbUSLPP/ZzIY= +github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= +github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= +github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= +github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= +github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= +github.com/jellydator/ttlcache/v3 v3.0.1 h1:cHgCSMS7TdQcoprXnWUptJZzyFsqs18Lt8VVhRuZYVU= +github.com/jellydator/ttlcache/v3 v3.0.1/go.mod h1:WwTaEmcXQ3MTjOm4bsZoDFiCu/hMvNWLO1w67RXz6h4= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= +github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= +github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= +github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= +github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= +github.com/kataras/iris/v12 v12.0.1/go.mod h1:udK4vLQKkdDqMGJJVd/msuMtN6hpYJhg/lSzuxjhO+U= +github.com/kataras/neffos v0.0.10/go.mod h1:ZYmJC07hQPW67eKuzlfY7SO3bC0mw83A3j6im82hfqw= +github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiDuX9AhMbDPkGYSPugBOV6yTZB1l2K9Z0= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= +github.com/klauspost/compress v1.17.1 h1:NE3C767s2ak2bweCZo3+rdP4U/HoyVXLv/X9f2gPS5g= +github.com/klauspost/compress v1.17.1/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +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/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= +github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= +github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= +github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= +github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= +github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= +github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= +github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= +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.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= +github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= +github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= +github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= +github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= +github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= +github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= +github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= +github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= +github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= +github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +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/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= +github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= +github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= +github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= +github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= +github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= +github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= +github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo= +github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= +github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= +github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= +github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= +github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= +github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= +github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= +github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= +github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +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/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= +github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 h1:m5ZsBa5o/0CkzZXfXLaThzKuR85SnHHetqBCpzQ30h8= +github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= +github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= +github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= +github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= +github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= +github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= +github.com/pingcap/kvproto v0.0.0-20230925123611-87bebcc0d071 h1:giqmIJSWHs+jhHfd+rth8CXWR18KAtqJu4imY1YdA6o= +github.com/pingcap/kvproto v0.0.0-20230925123611-87bebcc0d071/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= +github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= +github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= +github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= +github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= +github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873 h1:IOQPa3ItEg9RrCBgkLGO6VsKRh72LXAbdKiSfPuPQmw= +github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873/go.mod h1:yXjpyctIACAIDBLGwwRis4X8MDUO8sFS4QerAsOouAc= +github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4 h1:xrrIEHBBSeBuCL6mbyoMG4R4/ro+WpEql9XKFWtDXRw= +github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4/go.mod h1:yRkiqLFwIqibYg2P7h4bclHjHcJiIFRLKhGRyBcKYus= +github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f h1:NCiI4Wyu4GkViLGTu6cYcxt79LZ1SenBBQX1OwEV6Jg= +github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= +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/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= +github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3gMACTjAbMZBjXAqTOzOwFaj2Ld6cjeQ7Rig= +github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= +github.com/prometheus/client_golang v1.17.0 h1:rl2sfwZMtSthVU752MqfjQozy7blglC+1SOtjMAMh+Q= +github.com/prometheus/client_golang v1.17.0/go.mod h1:VeL+gMmOAxkS2IqfCq0ZmHSL+LjWfWDUmp1mBz9JgUY= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= +github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= +github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= +github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY= +github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= +github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= +github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec h1:W09IVJc94icq4NjY3clb7Lk8O1qJ8BdBEF8z0ibU0rE= +github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= +github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= +github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= +github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= +github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= +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/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= +github.com/segmentio/kafka-go v0.4.45 h1:prqrZp1mMId4kI6pyPolkLsH6sWOUmDxmmucbL4WS6E= +github.com/segmentio/kafka-go v0.4.45/go.mod h1:HjF6XbOKh0Pjlkr5GVZxt6CsjjwnmhVOfURM5KMd8qg= +github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= +github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= +github.com/shirou/gopsutil/v3 v3.23.10 h1:/N42opWlYzegYaVkWejXWJpbzKv2JDy3mrgGzKsh9hM= +github.com/shirou/gopsutil/v3 v3.23.10/go.mod h1:JIE26kpucQi+innVlAUnIEOSBhBUkirr5b44yr55+WE= +github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= +github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= +github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= +github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= +github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= +github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= +github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= +github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= +github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= +github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +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/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/thessem/zap-prettyconsole v0.3.0 h1:jreGIwOwkfqpA1NWTsbXXr0ZSL68b39KpH54q9Hz58U= +github.com/thessem/zap-prettyconsole v0.3.0/go.mod h1:93z1PhlPAYOWIOhPJvNkxE9cVM8jE5xB6/6HCtnvmcA= +github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= +github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= +github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e h1:kl8+gDOfPfRqkc1VDhhjhezMvsbfRENYsm/FqSIDnwg= +github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e/go.mod h1:fEAE7GS/lta+OasPOacdgy6RlJIRaq9/Cyr2WbSYcBE= +github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865 h1:Gkvo77EevOpBGIdV1c8gwRqPhVbgLPRy82tXNEFpGTc= +github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865/go.mod h1:cd6zBqRM9aogxf26K8NnFRPVtq9BnRE59tKEpX8IaWQ= +github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= +github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= +github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= +github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= +github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= +github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= +github.com/twmb/murmur3 v1.1.6 h1:mqrRot1BRxm+Yct+vavLMou2/iJt0tNVTTC0QoIjaZg= +github.com/twmb/murmur3 v1.1.6/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= +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 v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= +github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= +github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= +github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= +github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= +github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= +github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= +github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= +github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= +github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.2/go.mod h1:RT/sEzTbU5y00aCK8UOx6R7YryM0iF1N2MOmC3kKLN4= +github.com/xdg-go/stringprep v1.0.4/go.mod h1:mPGuuIYwz7CmR2bT9j4GbQqutWS1zV24gijq1dTyGkM= +github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= +github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= +github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= +github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= +github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= +github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= +github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= +github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= +github.com/yusufpapurcu/wmi v1.2.3 h1:E1ctvB7uKFMOJw3fdOW32DwGE9I7t++CRUEMKvFoFiw= +github.com/yusufpapurcu/wmi v1.2.3/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= +go.etcd.io/etcd/api/v3 v3.5.10 h1:szRajuUUbLyppkhs9K6BRtjY37l66XQQmw7oZRANE4k= +go.etcd.io/etcd/api/v3 v3.5.10/go.mod h1:TidfmT4Uycad3NM/o25fG3J07odo4GBB9hoxaodFCtI= +go.etcd.io/etcd/client/pkg/v3 v3.5.10 h1:kfYIdQftBnbAq8pUWFXfpuuxFSKzlmM5cSn76JByiT0= +go.etcd.io/etcd/client/pkg/v3 v3.5.10/go.mod h1:DYivfIviIuQ8+/lCq4vcxuseg2P2XbHygkKwFo9fc8U= +go.etcd.io/etcd/client/v3 v3.5.10 h1:W9TXNZ+oB3MCd/8UjxHTWK5J9Nquw9fQBLJd5ne5/Ao= +go.etcd.io/etcd/client/v3 v3.5.10/go.mod h1:RVeBnDz2PUEZqTpgqwAtUd8nAPf5kjyFyND7P1VkOKc= +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/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= +go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= +go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= +go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= +go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= +go.uber.org/mock v0.3.0 h1:3mUxI1No2/60yUYax92Pt8eNOEecx2D3lcXZh2NEZJo= +go.uber.org/mock v0.3.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc= +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/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= +go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= +go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= +go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= +go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= +go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= +go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= +go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= +go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= +go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= +go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= +go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= +golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= +golang.org/x/crypto v0.15.0 h1:frVn1TEaCEaZcn3Tmd7Y2b5KKPaZ+I32Q2OA3kYp5TA= +golang.org/x/crypto v0.15.0/go.mod h1:4ChreQoLWfG3xLDer1WdlH5NdlQ3+mwnQq1YTKY+72g= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20231006140011-7918f672742d h1:jtJma62tbqLibJ5sFQz8bKtEM8rJBtfilJ2qTU199MI= +golang.org/x/exp v0.0.0-20231006140011-7918f672742d/go.mod h1:ldy0pHrwJyGW56pPQzzkH36rKxoZW1tw7ZJpeKx+hdo= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/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-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.18.0 h1:mIYleuAkSbHh0tCv7RvjL3F6ZVbLjq4+R7zbOn3Kokg= +golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-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/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= +golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= +golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= +golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= +golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= +golang.org/x/tools v0.14.0 h1:jvNa2pY0M4r62jkRQ6RwEZZyPcymeL9XZMLBbV7U2nc= +golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b h1:+YaDE2r2OG8t/z5qmsh7Y+XXwCbvadxxZ0YY6mTdrVA= +google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:CgAqfJo+Xmu0GwA0411Ht3OU3OntXwsGmrmjI8ioGXI= +google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b h1:CIC2YMXmIhYw6evmhPxBKJ4fmLbOFtXQN/GV3XOZR8k= +google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:IBQ646DjkDkvUIsVq/cc03FUFQ9wbZu7yE396YcL870= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b h1:ZlWIi1wSK56/8hn4QcBp/j9M7Gt3U/3hZw3mC7vDICo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc= +google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= +google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= +google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/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/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= +gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= +gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= +gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= +gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= +gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= +gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYskCTPBJVb9jqSc= +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.2.2/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.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= diff --git a/tests/integration_tests/debezium/sql/data_types.sql b/tests/integration_tests/debezium/sql/data_types.sql new file mode 100644 index 00000000000..6d3096c2777 --- /dev/null +++ b/tests/integration_tests/debezium/sql/data_types.sql @@ -0,0 +1,401 @@ +SET sql_mode='strict_trans_tables'; +SET time_zone='+06:00'; + +/* +---------------------------------------------------------------------- +-- DATE type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_date( + col DATE, + pk INT PRIMARY KEY +); + +INSERT INTO t_date VALUES ('2023-11-16', 1); +INSERT INTO t_date VALUES ('1000-01-01', 2); +INSERT INTO t_date VALUES ('9999-12-31', 3); + +SET sql_mode=''; +INSERT INTO t_date VALUES (/* Zero dates */ '0000-00-00', 4); +INSERT INTO t_date VALUES (/* Invalid dates */ '2009-11-31', 5); +SET sql_mode='strict_trans_tables'; + +/* +---------------------------------------------------------------------- +-- DATETIME type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_datetime( + col DATETIME, + col_0 DATETIME(0), + col_1 DATETIME(1), + col_2 DATETIME(2), + col_3 DATETIME(3), + col_4 DATETIME(4), + col_5 DATETIME(5), + col_6 DATETIME(6), + col_z DATETIME DEFAULT 0, + -- TODO: + -- col_default_current_timestamp DATETIME DEFAULT CURRENT_TIMESTAMP, + pk INT PRIMARY KEY +); + +INSERT INTO t_datetime VALUES ( + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + NULL, + -- '2023-11-16 12:34:56.123456', + 1 +); + +INSERT INTO t_datetime VALUES ( + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + NULL, + -- '2023-11-16 12:34:56', + 2 +); + +SET time_zone='+04:00'; + +INSERT INTO t_datetime VALUES ( + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + NULL, + -- '2023-11-16 12:34:56.123456', + 3 +); + +SET time_zone='+06:00'; + +INSERT INTO t_datetime VALUES ( + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + -- NULL, + 4 +); + +/* +---------------------------------------------------------------------- +-- TIME type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_time( + col TIME, + col_0 TIME(0), + col_1 TIME(1), + col_5 TIME(5), + col_6 TIME(6), + pk INT PRIMARY KEY +); + +INSERT INTO t_time VALUES ( + '00:00:00', + '00:00:00', + '00:00:00', + '00:00:00', + '00:00:00', + 1 +); + +INSERT INTO t_time VALUES ( + '00:00:00.123456', + '00:00:00.123456', + '00:00:00.123456', + '00:00:00.123456', + '00:00:00.123456', + 2 +); + +INSERT INTO t_time VALUES ( + '-10:23:45.123456', + '-10:23:45.123456', + '-10:23:45.123456', + '-10:23:45.123456', + '-10:23:45.123456', + 3 +); + +/* + +Commented out because Debezium produce wrong result: + +"col":-3020399000000, +"col_0":-3020399000000, +"col_1":-3020400147483, +"col_5":-3020399048576, +"col_6":-3020399048576, + +INSERT INTO t_time VALUES ( + '-838:59:59.000000', + '-838:59:59.000000', + '-838:59:59.000000', + '-838:59:59.000000', + '-838:59:59.000000', + 3 +); + +*/ + +INSERT INTO t_time VALUES ( + '838:59:59.000000', + '838:59:59.000000', + '838:59:59.000000', + '838:59:59.000000', + '838:59:59.000000', + 4 +); + +INSERT INTO t_time VALUES ( + '123:44:56.123456', + '123:44:56.123456', + '123:44:56.123456', + '123:44:56.123456', + '123:44:56.123456', + 5 +); + +/* +---------------------------------------------------------------------- +-- TIMESTAMP type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_timestamp( + col TIMESTAMP, + col_0 TIMESTAMP(0), + col_1 TIMESTAMP(1), + col_5 TIMESTAMP(5), + col_6 TIMESTAMP(6), + col_z TIMESTAMP DEFAULT 0, + pk INT PRIMARY KEY +); + +INSERT INTO t_timestamp VALUES ( + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + NULL, + 1 +); + +INSERT INTO t_timestamp VALUES ( + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + '2023-11-16 12:34:56', + NULL, + 2 +); + +SET time_zone='+04:00'; + +INSERT INTO t_timestamp VALUES ( + '2023-11-16 12:34:56.123456', + FROM_UNIXTIME(1), + FROM_UNIXTIME(1470762668), + '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', + NULL, + 3 +); + +SET time_zone='+06:00'; + +/* +---------------------------------------------------------------------- +-- YEAR type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_year( + col YEAR, + col_4 YEAR(4), + pk INT PRIMARY KEY +); + +INSERT INTO t_year VALUES (1901, 1901, 1); + +/* +---------------------------------------------------------------------- +-- BIT type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_bit( + col_1 BIT(1), + col_5 BIT(5), + col_6 BIT(6), + col_60 BIT(60), + pk INT PRIMARY KEY +); + +INSERT INTO t_bit VALUES (0, 16, 16, 16, 1); +INSERT INTO t_bit VALUES (1, 1, 1, 1, 2); + +/* +---------------------------------------------------------------------- +-- VARCHAR type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_varchar( + col VARCHAR(64), + col_utf8_bin VARCHAR(64) + CHARACTER SET utf8mb4 COLLATE utf8mb4_bin, + /* + + Note: There is a bug of Debezium that it does not recognize + VARCHAR(..) CHARACTER SET BINARY as binary type + when the table is created AFTER the connector. + + col_bin VARCHAR(64) + CHARACTER SET BINARY, + */ + + pk INT PRIMARY KEY +); + +INSERT INTO t_varchar VALUES ('abc', 'abc', /* 'abc' , */ 1); +INSERT INTO t_varchar VALUES ('def', 'def', /* 0xAABBCC , */ 2); + +/* +---------------------------------------------------------------------- +-- BINARY / VARBINARY type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_binary( + col_1 BINARY(64), + col_2 VARBINARY(64), + pk INT PRIMARY KEY +); + +INSERT INTO t_binary VALUES ('abc', 'abc', 1); +INSERT INTO t_binary VALUES ('def', 'def', 2); + +/* +---------------------------------------------------------------------- +-- BLOB / TEXT type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_blob( + col_b BLOB, + col_t TEXT, + pk INT PRIMARY KEY +); + +INSERT INTO t_blob VALUES ('abc', 'abc', 1); + +/* +---------------------------------------------------------------------- +-- BOOL type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_bool( + /* + + We do not support BOOL type. + Debezium supports BOOL type only when the table is created AFTER the connector. + + col_bool BOOL, + + */ + + col_tinyint_1 TINYINT(1), + col_tinyint_1_u TINYINT(1) UNSIGNED, + col_tinyint_2 TINYINT(2), + pk INT PRIMARY KEY +); + +INSERT INTO t_bool VALUES(/* true, */ 10, 10, 10, 1); +INSERT INTO t_bool VALUES(/* false, */ 10, 10, 10, 2); + + +/* +---------------------------------------------------------------------- +-- FLOAT / DOUBLE type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_float( + col_f FLOAT, + col_d DOUBLE, + pk INT PRIMARY KEY +); + +INSERT INTO t_float VALUES (12345.12345, 12345.12345, 1); + +/* +---------------------------------------------------------------------- +-- DECIMAL / NUMERIC type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_decimal( + col_d DECIMAL(10, 5), + col_n NUMERIC(10, 5), + pk INT PRIMARY KEY +); + +INSERT INTO t_decimal VALUES (12345.12345, 12345.12345, 1); + +/* +---------------------------------------------------------------------- +-- JSON type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_json( + col JSON, + pk INT PRIMARY KEY +); + +INSERT INTO t_json VALUES ('["foo"]', 1); + +/* +---------------------------------------------------------------------- +-- ENUM / SET type +---------------------------------------------------------------------- +*/ + +CREATE TABLE t_enum( + col_e ENUM('a', 'b', 'c'), + col_s SET('a', 'b', 'c'), + pk INT PRIMARY KEY +); + +INSERT INTO t_enum VALUES ('a', 'c', 1); diff --git a/tests/integration_tests/debezium/sql/debezium/LICENSE.txt b/tests/integration_tests/debezium/sql/debezium/LICENSE.txt new file mode 100644 index 00000000000..d6456956733 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/tests/integration_tests/debezium/sql/debezium/README.txt b/tests/integration_tests/debezium/sql/debezium/README.txt new file mode 100644 index 00000000000..6979227b498 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/README.txt @@ -0,0 +1,3 @@ +Test files in this directory is ported from: + +https://github.com/debezium/debezium/tree/main/debezium-connector-mysql/src/test/resources/ddl diff --git a/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql b/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql new file mode 100644 index 00000000000..245452e0e27 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql @@ -0,0 +1,14 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: binary_column_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE dbz_254_binary_column_test ( + id INT AUTO_INCREMENT NOT NULL, + file_uuid BINARY(16), + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO dbz_254_binary_column_test VALUES (default, unhex(replace('651aed08-390f-4893-b2f1-36923e7b7400','-',''))); +INSERT INTO dbz_254_binary_column_test VALUES (default, unhex(replace('651aed08-390f-4893-b2f1-36923e7b74ab','-',''))); +INSERT INTO dbz_254_binary_column_test VALUES (default, unhex(replace('651aed08-390f-4893-b2f1-36923e7b74','-',''))); +INSERT INTO dbz_254_binary_column_test VALUES (default, unhex(00)); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql b/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql new file mode 100644 index 00000000000..8ad20ccad17 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql @@ -0,0 +1,31 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: binary_column_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE dbz_1814_binary_mode_test ( + id INT AUTO_INCREMENT NOT NULL, + blob_col BLOB NOT NULL, + tinyblob_col TINYBLOB NOT NULL, + mediumblob_col MEDIUMBLOB NOT NULL, + longblob_col LONGBLOB NOT NULL, + binary_col BINARY(3) NOT NULL, + varbinary_col varbinary(20) NOT NULL, + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO dbz_1814_binary_mode_test ( + id, + blob_col, + tinyblob_col, + mediumblob_col, + longblob_col, + binary_col, + varbinary_col ) +VALUES ( + default, + X'010203', + X'010203', + X'010203', + X'010203', + X'010203', + X'010203' ); diff --git a/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql b/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql new file mode 100644 index 00000000000..0878fd3ec34 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql @@ -0,0 +1,18 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: connector_read_binary_field_test +-- ---------------------------------------------------------------------------------------------------------------- + +-- Create a table, mainly MySQL time type fields +CREATE TABLE binary_field +( + id INT AUTO_INCREMENT PRIMARY KEY, + now_time TIME, + now_date DATE, + now_date_time DATETIME, + now_time_stamp TIMESTAMP +) ENGINE = innodb + AUTO_INCREMENT = 1 + DEFAULT CHARSET = utf8; + +INSERT INTO binary_field +VALUES (default, now(), now(), now(), now()); diff --git a/tests/integration_tests/debezium/sql/debezium/connector_test.sql b/tests/integration_tests/debezium/sql/debezium/connector_test.sql new file mode 100644 index 00000000000..d6534176e00 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/connector_test.sql @@ -0,0 +1,75 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: connector_test +-- ---------------------------------------------------------------------------------------------------------------- + +-- Create and populate our products using a single insert with many rows +CREATE TABLE products ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512), + weight FLOAT +); +ALTER TABLE products AUTO_INCREMENT = 101; + +INSERT INTO products +VALUES (default,"scooter","Small 2-wheel scooter",3.14), + (default,"car battery","12V car battery",8.1), + (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), + (default,"hammer","12oz carpenter's hammer",0.75), + (default,"hammer","14oz carpenter's hammer",0.875), + (default,"hammer","16oz carpenter's hammer",1.0), + (default,"rocks","box of assorted rocks",5.3), + (default,"jacket","water resistent black wind breaker",0.1), + (default,"spare tire","24 inch spare tire",22.2); + +-- Create and populate the products on hand using multiple inserts +CREATE TABLE products_on_hand ( + product_id INTEGER NOT NULL PRIMARY KEY, + quantity INTEGER NOT NULL, + FOREIGN KEY (product_id) REFERENCES products(id) +); + +INSERT INTO products_on_hand VALUES (101,3); +INSERT INTO products_on_hand VALUES (102,8); +INSERT INTO products_on_hand VALUES (103,18); +INSERT INTO products_on_hand VALUES (104,4); +INSERT INTO products_on_hand VALUES (105,5); +INSERT INTO products_on_hand VALUES (106,0); +INSERT INTO products_on_hand VALUES (107,44); +INSERT INTO products_on_hand VALUES (108,2); +INSERT INTO products_on_hand VALUES (109,5); + +-- Create some customers ... +CREATE TABLE customers ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + first_name VARCHAR(255) NOT NULL, + last_name VARCHAR(255) NOT NULL, + email VARCHAR(255) NOT NULL UNIQUE KEY +) AUTO_INCREMENT=1001; + + +INSERT INTO customers +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), + (default,"George","Bailey","gbailey@foobar.com"), + (default,"Edward","Walker","ed@walker.com"), + (default,"Anne","Kretchmar","annek@noanswer.org"); + +-- Create some very simple orders +CREATE TABLE orders ( + order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + order_date DATE NOT NULL, + purchaser INTEGER NOT NULL, + quantity INTEGER NOT NULL, + product_id INTEGER NOT NULL, + FOREIGN KEY order_customer (purchaser) REFERENCES customers(id), + FOREIGN KEY ordered_product (product_id) REFERENCES products(id) +) AUTO_INCREMENT = 10001; + +INSERT INTO orders +VALUES (default, '2016-01-16', 1001, 1, 102), + (default, '2016-01-17', 1002, 2, 105), + (default, '2016-02-18', 1004, 3, 109), + (default, '2016-02-19', 1002, 2, 106), + (default, '16-02-21', 1003, 1, 107); + +CREATE DATABASE IF NOT EXISTS emptydb; diff --git a/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql b/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql new file mode 100644 index 00000000000..22ab5d1dda6 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql @@ -0,0 +1,87 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: connector_test_ro +-- ---------------------------------------------------------------------------------------------------------------- + +-- Create and populate our products using a single insert with many rows +CREATE TABLE Products ( + PRIMARY KEY (id), + id INTEGER NOT NULL AUTO_INCREMENT, + name VARCHAR(255) NOT NULL, + description VARCHAR(512), + weight FLOAT +); +ALTER TABLE Products AUTO_INCREMENT = 101; + +INSERT INTO Products +VALUES (default,"scooter","Small 2-wheel scooter",3.14), + (default,"car battery","12V car battery",8.1), + (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), + (default,"hammer","12oz carpenter's hammer",0.75), + (default,"hammer2","14oz carpenter's hammer",8.75E-1), + (default,"hammer3","16oz carpenter's hammer",1.0), + (default,"rocks","box of assorted rocks",5.3), + (default,"jacket","water resistent black wind breaker",0.1), + (default,"spare tire","24 inch spare tire",22.2); + +-- Create and populate the products on hand using multiple inserts +CREATE TABLE products_on_hand ( + product_id INTEGER NOT NULL PRIMARY KEY, + quantity INTEGER NOT NULL, + FOREIGN KEY (product_id) REFERENCES Products(id) +); + +INSERT INTO products_on_hand VALUES (101,3); +INSERT INTO products_on_hand VALUES (102,8); +INSERT INTO products_on_hand VALUES (103,18); +INSERT INTO products_on_hand VALUES (104,4); +INSERT INTO products_on_hand VALUES (105,5); +INSERT INTO products_on_hand VALUES (106,0); +INSERT INTO products_on_hand VALUES (107,44); +INSERT INTO products_on_hand VALUES (108,2); +INSERT INTO products_on_hand VALUES (109,5); + +-- Create some customers ... +CREATE TABLE customers ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + first_name VARCHAR(255) NOT NULL, + last_name VARCHAR(255) NOT NULL, + email VARCHAR(255) NOT NULL UNIQUE KEY +) AUTO_INCREMENT=1001; + + +INSERT INTO customers +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), + (default,"George","Bailey","gbailey@foobar.com"), + (default,"Edward","Walker","ed@walker.com"), + (default,"Anne","Kretchmar","annek@noanswer.org"); + +-- Create some very simple orders +CREATE TABLE orders ( + order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + order_date DATE NOT NULL, + purchaser INTEGER NOT NULL, + quantity INTEGER NOT NULL, + product_id INTEGER NOT NULL, + FOREIGN KEY order_customer (purchaser) REFERENCES customers(id), + FOREIGN KEY ordered_product (product_id) REFERENCES Products(id) +) AUTO_INCREMENT = 10001; + +INSERT INTO orders +VALUES (default, '2016-01-16', 1001, 1, 102), + (default, '2016-01-17', 1002, 2, 105), + (default, '2016-02-18', 1004, 3, 109), + (default, '2016-02-19', 1002, 2, 106), + (default, '2016-02-21', 1003, 1, 107); + + +-- DBZ-342 handle TIME values that exceed the value range of java.sql.Time +CREATE TABLE dbz_342_timetest ( + c1 TIME(2) PRIMARY KEY, + c2 TIME(0), + c3 TIME(3), + c4 TIME(3), + c5 TIME(6) +); +INSERT INTO dbz_342_timetest VALUES ('517:51:04.777', '-13:14:50', '-733:00:00.0011', '-1:59:59.0011', '-838:59:58.999999'); + +CREATE DATABASE IF NOT EXISTS emptydb; diff --git a/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql b/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql new file mode 100644 index 00000000000..08740721ba3 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql @@ -0,0 +1,14 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: datetime_key_test +-- ---------------------------------------------------------------------------------------------------------------- + +SET sql_mode=''; +CREATE TABLE dbz_1194_datetime_key_test ( + id INT AUTO_INCREMENT NOT NULL, + dtval DATETIME NOT NULL, + dval DATE NOT NULL, + tval TIME NOT NULL, + PRIMARY KEY (id, dtval, dval, tval) +) DEFAULT CHARSET=utf8; + +INSERT INTO dbz_1194_datetime_key_test VALUES (default, '0000-00-00 00:00:00', '0000-00-00', '00:00:00'); diff --git a/tests/integration_tests/debezium/sql/debezium/db_default_charset.sql b/tests/integration_tests/debezium/sql/debezium/db_default_charset.sql new file mode 100644 index 00000000000..764da8e67e5 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/db_default_charset.sql @@ -0,0 +1,5 @@ +CREATE TABLE DATA ( + MESSAGE TEXT, + PK INT PRIMARY KEY +); +INSERT INTO DATA VALUES ('Žluťoučký', 1); diff --git a/tests/integration_tests/debezium/sql/debezium/db_default_charset_noutf.sql b/tests/integration_tests/debezium/sql/debezium/db_default_charset_noutf.sql new file mode 100644 index 00000000000..46610614847 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/db_default_charset_noutf.sql @@ -0,0 +1,6 @@ +CREATE TABLE DATA ( + MESSAGE TEXT, + FLAG TINYINT(1), + PK INT PRIMARY KEY +); +INSERT INTO DATA VALUES ('Žluťoučký', 1, 1); diff --git a/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql b/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql new file mode 100644 index 00000000000..64cd7742ad5 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql @@ -0,0 +1,14 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: decimal_column_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE dbz_751_decimal_column_test ( + id INT AUTO_INCREMENT NOT NULL, + rating1 DECIMAL, + rating2 DECIMAL(8, 4), + rating3 DECIMAL(7), + rating4 DECIMAL(6, 0), + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO dbz_751_decimal_column_test VALUES (default, 123, 123.4567, 234.5, 345.6); diff --git a/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql b/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql new file mode 100644 index 00000000000..5b6320725a5 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql @@ -0,0 +1,31 @@ +CREATE TABLE all_zero_date_and_time_table ( + A TIMESTAMP NOT NULL DEFAULT '0000-00-00 00:00:00', + B TIMESTAMP NULL DEFAULT '0000-00-00 00:00:00', + C TIMESTAMP DEFAULT '0000-00-00 00:00:00' NOT NULL, + D TIMESTAMP DEFAULT '0000-00-00 00:00:00' NULL, + E DATE NOT NULL DEFAULT '0000-00-00', + F DATE NULL DEFAULT '0000-00-00', + G DATE DEFAULT '0000-00-00' NOT NULL, + H DATE DEFAULT '0000-00-00' NULL, + I DATETIME NOT NULL DEFAULT '0000-00-00 00:00:00', + J DATETIME NULL DEFAULT '0000-00-00 00:00:00', + K DATETIME DEFAULT '0000-00-00 00:00:00' NOT NULL, + L DATETIME DEFAULT '0000-00-00 00:00:00' NULL, + PK INT PRIMARY KEY +); +INSERT INTO all_zero_date_and_time_table +VALUES (DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, 1 ); + +CREATE TABLE part_zero_date_and_time_table ( + A TIMESTAMP NOT NULL DEFAULT '0000-00-00 00:00:00', + B TIMESTAMP NULL DEFAULT '0000-00-00 00:00:00', + C DATETIME NOT NULL DEFAULT '0000-00-00 01:00:00.000', + D DATETIME NULL DEFAULT '0000-00-00 01:00:00.000', + E DATE NOT NULL DEFAULT '1000-00-00 01:00:00.000', + F DATE NULL DEFAULT '1000-00-00 01:00:00.000', + G TIME DEFAULT '0000-00-00 01:00:00.000' NOT NULL, + H TIME DEFAULT '0000-00-00 01:00:00.000' NULL, + PK INT PRIMARY KEY +); +INSERT INTO part_zero_date_and_time_table +VALUES (DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, 1 ); diff --git a/tests/integration_tests/debezium/sql/debezium/default_value_generated.sql b/tests/integration_tests/debezium/sql/debezium/default_value_generated.sql new file mode 100644 index 00000000000..46bca73985a --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/default_value_generated.sql @@ -0,0 +1,7 @@ +CREATE TABLE GENERATED_TABLE ( + A SMALLINT UNSIGNED, + B SMALLINT UNSIGNED AS (2 * A), + C SMALLINT UNSIGNED AS (3 * A) NOT NULL, + PK INT PRIMARY KEY +); +INSERT INTO GENERATED_TABLE VALUES (15, DEFAULT, DEFAULT, 1); diff --git a/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql b/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql new file mode 100644 index 00000000000..a8db5dfe308 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql @@ -0,0 +1,16 @@ +CREATE TABLE `test_stations_10` ( + `id` int(10) unsigned NOT NULL AUTO_INCREMENT, + `name` varchar(500) COLLATE utf8_unicode_ci NOT NULL, + `type` enum('station', 'post_office') COLLATE utf8_unicode_ci NOT NULL DEFAULT 'station', + `created` datetime DEFAULT CURRENT_TIMESTAMP, + `modified` datetime DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, + PRIMARY KEY (`id`) +); + +INSERT INTO test_stations_10 (`name`, `type`) values ( 'ha Tinh 7', 'station' ); + +ALTER TABLE `test_stations_10` + MODIFY COLUMN `type` ENUM('station', 'post_office', 'plane', 'ahihi_dongok', 'now', 'test', 'a,b', 'c,\'d', 'g,''h') + CHARACTER SET 'utf8' COLLATE 'utf8_unicode_ci' NOT NULL DEFAULT 'station'; + +INSERT INTO test_stations_10 ( `name`, `type` ) values ( 'Ha Tinh 1', 'now' ); diff --git a/tests/integration_tests/debezium/sql/debezium/json_test.sqlx b/tests/integration_tests/debezium/sql/debezium/json_test.sqlx new file mode 100644 index 00000000000..336dd038928 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/json_test.sqlx @@ -0,0 +1,137 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: json_test +-- ---------------------------------------------------------------------------------------------------------------- +-- The integration test for this database expects to scan all of the binlog events associated with this database +-- without error or problems. The integration test does not modify any records in this database, so this script +-- must contain all operations to these tables. +-- +-- This relies upon MySQL 5.7's JSON datatype. + +-- DBZ-126 handle JSON column types ... +CREATE TABLE dbz_126_jsontable ( + id INT AUTO_INCREMENT NOT NULL, + json JSON, + expectedJdbcStr VARCHAR(256), -- value that we get back from JDBC + expectedBinlogStr VARCHAR(256), -- value we parse from the binlog + PRIMARY KEY(id) +) DEFAULT CHARSET=utf8; +INSERT INTO dbz_126_jsontable VALUES (default,NULL, + NULL, + NULL); +INSERT INTO dbz_126_jsontable VALUES (default,'{"a": 2}', + '{"a": 2}', + '{"a":2}'); +INSERT INTO dbz_126_jsontable VALUES (default,'[1, 2]', + '[1, 2]', + '[1,2]'); +INSERT INTO dbz_126_jsontable VALUES (default,'{"key1": "value1", "key2": "value2"}', + '{"key1": "value1", "key2": "value2"}', + '{"key1":"value1","key2":"value2"}'); +INSERT INTO dbz_126_jsontable VALUES (default,'["a", "b",1]', + '["a", "b",1]', + '["a","b",1]'); +INSERT INTO dbz_126_jsontable VALUES (default,'{"k1": "v1", "k2": {"k21": "v21", "k22": "v22"}, "k3": ["a", "b", 1]}', + '{"k1": "v1", "k2": {"k21": "v21", "k22": "v22"}, "k3": ["a", "b", 1]}', + '{"k1":"v1","k2":{"k21":"v21","k22":"v22"},"k3":["a","b",1]}'); +INSERT INTO dbz_126_jsontable VALUES (default,'{"a": "b", "c": "d", "ab": "abc", "bc": ["x", "y"]}', + '{"a": "b", "c": "d", "ab": "abc", "bc": ["x", "y"]}', + '{"a":"b","c":"d","ab":"abc","bc":["x","y"]}'); +INSERT INTO dbz_126_jsontable VALUES (default,'["here", ["I", "am"], "!!!"]', + '["here", ["I", "am"], "!!!"]', + '["here",["I","am"],"!!!"]'); +INSERT INTO dbz_126_jsontable VALUES (default,'"scalar string"', + '"scalar string"', + '"scalar string"'); +INSERT INTO dbz_126_jsontable VALUES (default,'true', + 'true', + 'true'); +INSERT INTO dbz_126_jsontable VALUES (default,'false', + 'false', + 'false'); +INSERT INTO dbz_126_jsontable VALUES (default,'null', + 'null', + 'null'); +INSERT INTO dbz_126_jsontable VALUES (default,'-1', + '-1', + '-1'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST(1 AS UNSIGNED) AS JSON), + '1', + '1'); +INSERT INTO dbz_126_jsontable VALUES (default,'32767', + '32767', + '32767'); +INSERT INTO dbz_126_jsontable VALUES (default,'32768', + '32768', + '32768'); +INSERT INTO dbz_126_jsontable VALUES (default,'-32768', + '-32768', + '-32768'); +INSERT INTO dbz_126_jsontable VALUES (default,'2147483647', -- INT32 + '2147483647', + '2147483647'); +INSERT INTO dbz_126_jsontable VALUES (default,'2147483648', -- INT64 + '2147483648', + '2147483648'); +INSERT INTO dbz_126_jsontable VALUES (default,'-2147483648', -- INT32 + '-2147483648', + '-2147483648'); +INSERT INTO dbz_126_jsontable VALUES (default,'-2147483649', -- INT64 + '-2147483649', + '-2147483649'); +INSERT INTO dbz_126_jsontable VALUES (default,'18446744073709551615', -- INT64 + '18446744073709551615', + '18446744073709551615'); +INSERT INTO dbz_126_jsontable VALUES (default,'18446744073709551616', -- BigInteger + '18446744073709551616', + '18446744073709551616'); +INSERT INTO dbz_126_jsontable VALUES (default,'3.14', + '3.14', + '3.14'); +INSERT INTO dbz_126_jsontable VALUES (default,'{}', + '{}', + '{}'); +INSERT INTO dbz_126_jsontable VALUES (default,'[]', + '[]', + '[]'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('2015-01-15 23:24:25' AS DATETIME) AS JSON), + '"2015-01-15 23:24:25"', + '"2015-01-15 23:24:25"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('2015-01-15 23:24:25.12' AS DATETIME(3)) AS JSON), + '"2015-01-15 23:24:25.12"', + '"2015-01-15 23:24:25.12"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('2015-01-15 23:24:25.0237' AS DATETIME(3)) AS JSON), + '"2015-01-15 23:24:25.024"', + '"2015-01-15 23:24:25.024"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('23:24:25' AS TIME) AS JSON), + '"23:24:25"', + '"23:24:25"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('23:24:25.12' AS TIME(3)) AS JSON), + '"23:24:25.12"', + '"23:24:25.12"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('23:24:25.0237' AS TIME(3)) AS JSON), + '"23:24:25.024"', + '"23:24:25.024"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('2015-01-15' AS DATE) AS JSON), + '"2015-01-15"', + '"2015-01-15"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(TIMESTAMP'2015-01-15 23:24:25' AS JSON), + '"2015-01-15 23:24:25"', + '"2015-01-15 23:24:25"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(TIMESTAMP'2015-01-15 23:24:25.12' AS JSON), + '"2015-01-15 23:24:25.12"', + '"2015-01-15 23:24:25.12"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(TIMESTAMP'2015-01-15 23:24:25.0237' AS JSON), + '"2015-01-15 23:24:25.0237"', + '"2015-01-15 23:24:25.0237"'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(UNIX_TIMESTAMP('2015-01-15 23:24:25') AS JSON), + '1421364265', + '1421364265'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST('[]' AS CHAR CHARACTER SET 'ascii'), + '[]', + '[]'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(x'cafe' AS JSON), -- BLOB as Base64 + '"yv4="', + '"yv4="'); +INSERT INTO dbz_126_jsontable VALUES (default,CAST(x'cafebabe' AS JSON), -- BLOB as Base64 + '"yv66vg=="', + '"yv66vg=="'); diff --git a/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql b/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql new file mode 100644 index 00000000000..679bb0169dc --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql @@ -0,0 +1,10 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: multitable_statement +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE t1 (ID INT PRIMARY KEY); +CREATE TABLE t2 (ID INT PRIMARY KEY); +CREATE TABLE t3 (ID INT PRIMARY KEY); +CREATE TABLE t4 (ID INT PRIMARY KEY); + +DROP TABLE t1,t2,t3,t4; \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/mysql_dbz_6533.sql b/tests/integration_tests/debezium/sql/debezium/mysql_dbz_6533.sql new file mode 100644 index 00000000000..f51396ebe7f --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql_dbz_6533.sql @@ -0,0 +1,18 @@ +CREATE TABLE tablename_suffix ( + PRIMARY KEY (id), + id INTEGER NOT NULL AUTO_INCREMENT +); + +CREATE TABLE tablename ( + PRIMARY KEY (id), + id INTEGER NOT NULL AUTO_INCREMENT +); + +CREATE TABLE another ( + PRIMARY KEY (id), + id INTEGER NOT NULL AUTO_INCREMENT +); + +INSERT INTO tablename_suffix VALUES (default); +INSERT INTO tablename VALUES (default); +INSERT INTO another VALUES (default); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql b/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql new file mode 100644 index 00000000000..48cf6071c43 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql @@ -0,0 +1,8 @@ +CREATE TABLE `NC_TEST` ( + `id` INT NOT NULL AUTO_INCREMENT, + `nc1` nchar default null, + `nc2` nchar(5) default null, + `nc3` nvarchar(25) default null, + PRIMARY KEY (`ID`) +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8mb4; +INSERT INTO NC_TEST(nc1,nc2,nc3) VALUES ('a', '123', 'hello'); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql b/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql new file mode 100644 index 00000000000..6b60b39b928 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql @@ -0,0 +1,14 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: numeric_column_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE dbz_751_numeric_column_test ( + id INT AUTO_INCREMENT NOT NULL, + rating1 NUMERIC, + rating2 NUMERIC(8, 4), + rating3 NUMERIC(7), + rating4 NUMERIC(6, 0), + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO dbz_751_numeric_column_test VALUES (default, 123, 123.4567, 234.5, 345.6); diff --git a/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql b/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql new file mode 100644 index 00000000000..7d6da0e272f --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql @@ -0,0 +1,23 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: readbinlog_test +-- Database needs to be populated to break dependency between MetadataIT and MySqlConnectorIT.shouldValidateAcceptableConfiguration run order +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE person ( + name VARCHAR(255) primary key, + birthdate DATE NULL, + age INTEGER NULL DEFAULT 10, + salary DECIMAL(5,2), bitStr BIT(18) +); +CREATE TABLE product ( + id INT NOT NULL AUTO_INCREMENT, + createdByDate DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP, + modifiedDate DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, + PRIMARY KEY(id) +); +CREATE TABLE purchased ( + purchaser VARCHAR(255) NOT NULL, + productId INT NOT NULL, + purchaseDate DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP, + PRIMARY KEY(productId,purchaser) +); diff --git a/tests/integration_tests/debezium/sql/debezium/real_test.sql b/tests/integration_tests/debezium/sql/debezium/real_test.sql new file mode 100644 index 00000000000..7fcf64f78ff --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/real_test.sql @@ -0,0 +1,6 @@ +CREATE TABLE `REAL_TEST` ( + `id` INT NOT NULL AUTO_INCREMENT, + `r1` real default 1.25, + PRIMARY KEY (`ID`) +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8mb4; +INSERT INTO REAL_TEST(r1) VALUE (2.36); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/regression_test.sql b/tests/integration_tests/debezium/sql/debezium/regression_test.sql new file mode 100644 index 00000000000..42f07803169 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/regression_test.sql @@ -0,0 +1,180 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: regression_test +-- ---------------------------------------------------------------------------------------------------------------- +-- The integration test for this database expects to scan all of the binlog events associated with this database +-- without error or problems. The integration test does not modify any records in this database, so this script +-- must contain all operations to these tables. + +-- DBZ-61 handle binary value recorded as hex string value +CREATE TABLE t1464075356413_testtable6 ( + pk_column int auto_increment NOT NULL, + varbinary_col varbinary(20) NOT NULL, + PRIMARY KEY(pk_column) +); +INSERT INTO t1464075356413_testtable6 (pk_column, varbinary_col) +VALUES(default, 0x4D7953514C); + +-- DBZ-84 Handle TINYINT +CREATE TABLE dbz84_integer_types_table ( + -- The column lengths are used for display purposes, and do not affect the range of values + colTinyIntA tinyint NOT NULL DEFAULT 100, + colTinyIntB tinyint(1) NOT NULL DEFAULT 101, + colTinyIntC tinyint(2) UNSIGNED NOT NULL DEFAULT 102, + colTinyIntD tinyint(3) UNSIGNED NOT NULL DEFAULT 103, + colSmallIntA smallint NOT NULL DEFAULT 200, + colSmallIntB smallint(1) NOT NULL DEFAULT 201, + colSmallIntC smallint(2) NOT NULL DEFAULT 201, + colSmallIntD smallint(3) NOT NULL DEFAULT 201, + colMediumIntA mediumint NOT NULL DEFAULT 300, + colMediumIntB mediumint(1) NOT NULL DEFAULT 301, + colMediumIntC mediumint(2) NOT NULL DEFAULT 302, + colMediumIntD mediumint(3) NOT NULL DEFAULT 303, + colIntA int NOT NULL DEFAULT 400, + colIntB int(1) NOT NULL DEFAULT 401, + colIntC int(2) NOT NULL DEFAULT 402, + colIntD int(3) NOT NULL DEFAULT 403, + colBigIntA bigint NOT NULL DEFAULT 500, + colBigIntB bigint(1) NOT NULL DEFAULT 501, + colBigIntC bigint(2) NOT NULL DEFAULT 502, + colBigIntD bigint(3) NOT NULL DEFAULT 503, + PK INT PRIMARY KEY +); +INSERT INTO dbz84_integer_types_table +VALUES(127,-128,128,255, default,201,202,203, default,301,302,303, default,401,402,403, default,501,502,503, 1); + +-- DBZ-85 handle fractional part of seconds +CREATE TABLE dbz_85_fractest ( + c1 DATE, + c2 TIME(2), + c3 DATETIME(2), + c4 TIMESTAMP(2), + PK INT PRIMARY KEY +); +INSERT INTO dbz_85_fractest VALUES ('2014-09-08', '17:51:04.777', '2014-09-08 17:51:04.777', '2014-09-08 17:51:04.777', 1); + +-- DBZ-100 handle enum and set +CREATE TABLE dbz_100_enumsettest ( + c1 enUM('a','b','c'), + c2 Set('a','b','c'), + PK INT PRIMARY KEY +); +INSERT INTO dbz_100_enumsettest VALUES ('a', 'a,b,c', 1); +INSERT INTO dbz_100_enumsettest VALUES ('b', 'b,a', 2); +INSERT INTO dbz_100_enumsettest VALUES ('c', 'a', 3); + +-- DBZ-102 handle character sets +-- Use session variables to dictate the character sets used by the client running these commands so +-- the literal value is interpretted correctly... +set character_set_client=utf8; +set character_set_connection=utf8; +CREATE TABLE dbz_102_charsettest ( + id INT(11) NOT NULL AUTO_INCREMENT, + text VARCHAR(255) DEFAULT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=2001 DEFAULT CHARSET=utf8; +INSERT INTO dbz_102_charsettest VALUES (default, "产品"); + +-- DBZ-114 handle zero-value dates +CREATE TABLE dbz_114_zerovaluetest ( + c1 DATE, + c2 TIME(2), + c3 DATETIME(2), + c4 TIMESTAMP(2), + PK INT PRIMARY KEY +); +INSERT IGNORE INTO dbz_114_zerovaluetest VALUES ('0000-00-00', '00:00:00.000', '0000-00-00 00:00:00.000', '0000-00-00 00:00:00.000', 1); +INSERT IGNORE INTO dbz_114_zerovaluetest VALUES ('0001-00-00', '00:01:00.000', '0001-00-00 00:00:00.000', '0001-00-00 00:00:00.000', 2); + +-- DBZ-1318 handle zero-value dates in when zero dates allowed +CREATE TABLE dbz_1318_zerovaluetest ( + c1 DATE, + c2 TIME(2), + c3 DATETIME(2), + c4 TIMESTAMP(2), + nnc1 DATE NOT NULL, + nnc2 TIME(2) NOT NULL, + nnc3 DATETIME(2) NOT NULL, + PK INT PRIMARY KEY +); +set sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION'; +INSERT IGNORE INTO dbz_1318_zerovaluetest VALUES ('0000-00-00', '00:00:00.000', '0000-00-00 00:00:00.000', '0000-00-00 00:00:00.000', '0000-00-00', '00:00:00.000', '0000-00-00 00:00:00.000', 1); +INSERT IGNORE INTO dbz_1318_zerovaluetest VALUES ('0001-00-00', '00:01:00.000', '0000-00-00 14:02:10.000', '0000-00-00 14:02:10.000', '0001-00-00', '00:01:00.000', '0000-00-00 14:02:10.000', 2); +set sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION'; + +-- DBZ-123 handle bit values, including bit field literals +CREATE TABLE dbz_123_bitvaluetest ( + c1 BIT, + c2 BIT(2), + c3 BIT(8) NOT NULL, + c4 BIT(64), + PK INT PRIMARY KEY +); +INSERT INTO dbz_123_bitvaluetest VALUES (1,2,64,23989979, 1); +INSERT INTO dbz_123_bitvaluetest VALUES (b'1',b'10',b'01000000',b'1011011100000111011011011', 2); + +-- DBZ-104 handle create table like ... +DROP DATABASE IF EXISTS connector_test; +CREATE DATABASE connector_test; +CREATE TABLE connector_test.customers ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + first_name VARCHAR(255) NOT NULL, + last_name VARCHAR(255) NOT NULL, + email VARCHAR(255) NOT NULL UNIQUE KEY +) AUTO_INCREMENT=1001; +INSERT INTO connector_test.customers +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), + (default,"George","Bailey","gbailey@foobar.com"), + (default,"Edward","Walker","ed@walker.com"), + (default,"Anne","Kretchmar","annek@noanswer.org"); + +CREATE TABLE dbz_104_customers LIKE connector_test.customers; +INSERT INTO dbz_104_customers SELECT * FROM connector_test.customers; + +-- DBZ-147 handle decimal value +CREATE TABLE dbz_147_decimalvalues ( + pk_column int auto_increment NOT NULL, + decimal_value decimal(7,2) NOT NULL, + PRIMARY KEY(pk_column) +); +INSERT INTO dbz_147_decimalvalues (pk_column, decimal_value) +VALUES(default, 12345.67); + +/* +-- DBZ-162 handle function declarations with newline characters +CREATE FUNCTION fnDbz162( p_creditLimit DOUBLE ) RETURNS VARCHAR(10) + DETERMINISTIC +BEGIN + DECLARE lvl VARCHAR(10)$$ + IF p_creditLimit > 50000 THEN + SET lvl = 'PLATINUM'$$ + ELSEIF (p_creditLimit <= 50000 AND p_creditLimit >= 10000) THEN + SET lvl = 'GOLD'$$ + ELSEIF p_creditLimit < 10000 THEN + SET lvl = 'SILVER'$$ + END IF$$ + RETURN (lvl)$$ +END$$ +; +*/ + +-- DBZ-195 handle numeric values +CREATE TABLE dbz_195_numvalues ( + id int auto_increment NOT NULL, + `search_version_read` int(11) NOT NULL DEFAULT '0', -- (11) is the display width + PRIMARY KEY (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=4972 DEFAULT CHARSET=utf8; + +INSERT INTO dbz_195_numvalues VALUES (default,0); +INSERT INTO dbz_195_numvalues VALUES (default,-2147483648); +INSERT INTO dbz_195_numvalues VALUES (default,2147483647); + +-- DBZ-342 handle TIME values that exceed the value range of java.sql.Time +CREATE TABLE dbz_342_timetest ( + c1 TIME(2), + c2 TIME(0), + c3 TIME(3), + c4 TIME(3), + c5 TIME(6), + PK INT PRIMARY KEY +); +INSERT INTO dbz_342_timetest VALUES ('517:51:04.777', '-13:14:50', '-733:00:00.0011', '-1:59:59.0011', '-838:59:58.999999', 1); diff --git a/tests/integration_tests/debezium/sql/debezium/skip_messages_test.sql b/tests/integration_tests/debezium/sql/debezium/skip_messages_test.sql new file mode 100644 index 00000000000..b2260b71be6 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/skip_messages_test.sql @@ -0,0 +1,6 @@ +CREATE TABLE `debezium_test` ( + id INT NOT NULL PRIMARY KEY, + black INT, + white INT +) ENGINE=InnoDB AUTO_INCREMENT=15851 DEFAULT CHARSET=utf8; +INSERT INTO `debezium_test`(id, black, white) VALUES (0,0,0); diff --git a/tests/integration_tests/debezium/sql/debezium/source_type_as_schema_parameter_test.sql b/tests/integration_tests/debezium/sql/debezium/source_type_as_schema_parameter_test.sql new file mode 100644 index 00000000000..d4bd690360c --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/source_type_as_schema_parameter_test.sql @@ -0,0 +1,16 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: source_type_as_schema_parameter_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE dbz_644_source_type_mapped_as_schema_parameter_test ( + id INT AUTO_INCREMENT NOT NULL, + c1 INT, + c2 MEDIUMINT, + c3a NUMERIC(5,2), + c3b VARCHAR(128), + f1 FLOAT(10), + f2 FLOAT(8,4), + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO dbz_644_source_type_mapped_as_schema_parameter_test VALUES (default, 123, 456, 789.01, 'test', 1.228, 234.56); diff --git a/tests/integration_tests/debezium/sql/debezium/strategy_test.sql b/tests/integration_tests/debezium/sql/debezium/strategy_test.sql new file mode 100644 index 00000000000..a7ba9e50f51 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/strategy_test.sql @@ -0,0 +1,39 @@ +CREATE TABLE `dbz4180` ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, + b DECIMAL(10, 3) NOT NULL DEFAULT 2.321, + c VARCHAR(255) NULL DEFAULT 'default mysql strategy', + d INT NULL DEFAULT '100', + e DATETIME NULL DEFAULT CURRENT_TIMESTAMP +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; +INSERT INTO `dbz4180`(a, b, c, d) +VALUES (1.33, -2.111 , 'topic strategy', 99); + +CREATE TABLE `dbz_4180_00` ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, + b DECIMAL(10, 3) NOT NULL DEFAULT 2.321, + c VARCHAR(255) NULL DEFAULT 'shard 0', + d INT NULL DEFAULT '100', + e DATETIME NULL DEFAULT CURRENT_TIMESTAMP +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; + +CREATE TABLE `dbz_4180_01` ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, + b DECIMAL(10, 3) NOT NULL DEFAULT 2.321, + c VARCHAR(255) NULL DEFAULT 'shard 1', + d INT NULL DEFAULT '100', + e DATETIME NULL DEFAULT CURRENT_TIMESTAMP +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; + +CREATE TABLE `dbz5743中文` ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, + b DECIMAL(10, 3) NOT NULL DEFAULT 2.321, + c VARCHAR(255) NULL DEFAULT 'default mysql strategy', + d INT NULL DEFAULT '100', + e DATETIME NULL DEFAULT CURRENT_TIMESTAMP +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; +INSERT INTO `dbz5743中文`(a, b, c, d) +VALUES (1.33, -2.111 , 'topic strategy', 99); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql b/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql new file mode 100644 index 00000000000..fc7b80aac2a --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql @@ -0,0 +1,12 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: table_column_comment_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE dbz_4000_comment_test ( + id INT AUTO_INCREMENT NOT NULL COMMENT 'pk', + name VARCHAR(255) NOT NULL COMMENT 'this is name column', + value BIGINT NULL COMMENT 'the value is bigint type', + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8 COMMENT='table for dbz-4000'; + +INSERT INTO dbz_4000_comment_test VALUES (default, 'DBZ-4000', 4000); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql b/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql new file mode 100644 index 00000000000..c80182cb59f --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql @@ -0,0 +1,15 @@ +CREATE TABLE t_user_black_list ( + `id` int(10) unsigned NOT NULL, + `data` varchar(20), + `create_time` datetime, + `update_time` datetime, + PRIMARY KEY (`id`) +); + +ALTER TABLE t_user_black_list + MODIFY COLUMN `update_time` datetime(0) NOT NULL + DEFAULT CURRENT_TIMESTAMP(0) COMMENT 'update_time' AFTER create_time; + +INSERT INTO t_user_black_list (`id`,`create_time`,`data`) VALUES (1, CURRENT_TIMESTAMP(), 'test'); + +UPDATE t_user_black_list SET `data` = 'test2' WHERE `id` = 1; \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql b/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql new file mode 100644 index 00000000000..76fd26f47f2 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql @@ -0,0 +1,23 @@ +CREATE TABLE `DBZ1773` ( + id int not null primary key auto_increment, + ti TINYINT SIGNED, + ti1 TINYINT(1) SIGNED, + ti2 TINYINT(2), + b BOOLEAN +) ENGINE=InnoDB AUTO_INCREMENT=10851 DEFAULT CHARSET=utf8; + +CREATE TABLE `DBZ2085` ( + id int not null primary key auto_increment, + b BOOLEAN NOT NULL DEFAULT 0 +) ENGINE=InnoDB AUTO_INCREMENT=10851 DEFAULT CHARSET=utf8; + +CREATE TABLE `DBZ5236` ( + id int(11) not null primary key auto_increment, + ti1 tinyint(3) unsigned NOT NULL DEFAULT '0', + ti2 tinyint(1) unsigned NOT NULL DEFAULT '0', + ti3 tinyint(1) NOT NULL DEFAULT '1' +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; + +INSERT INTO DBZ1773 VALUES (DEFAULT, 100, 5, 50, TRUE); +INSERT INTO DBZ2085 VALUES (DEFAULT, 1); +INSERT INTO DBZ5236 VALUES (DEFAULT, 1, 1, 0); diff --git a/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql b/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql new file mode 100644 index 00000000000..878cb7cb2a2 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql @@ -0,0 +1,19 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: topic-name.sanitization-it +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE `dbz_878_some|test@data` ( + id INT, + some_col VARCHAR(255), + PRIMARY KEY (id) + ) DEFAULT CHARSET=utf8; + +INSERT INTO `dbz_878_some|test@data` VALUES (1, 'some text'); + +CREATE TABLE `DBZ.1834` ( + id INT, + some_col VARCHAR(255), + PRIMARY KEY (id) + ) DEFAULT CHARSET=utf8; + +INSERT INTO `DBZ.1834` VALUES (1, 'some text'); diff --git a/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql b/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql new file mode 100644 index 00000000000..1ee023a636d --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql @@ -0,0 +1,92 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: unsigned_integer_test +-- ---------------------------------------------------------------------------------------------------------------- +-- The integration test for this database expects to scan all of the binlog events associated with this database +-- without error or problems. The integration test does not modify any records in this database, so this script +-- must contain all operations to these tables. +-- +-- This relies upon MySQL 5.7's Geometries datatypes. + +-- DBZ-228 handle unsigned TINYINT UNSIGNED +CREATE TABLE dbz_228_tinyint_unsigned ( + id int auto_increment NOT NULL, + c1 TINYINT(3) UNSIGNED ZEROFILL NOT NULL, + c2 TINYINT(3) UNSIGNED NOT NULL, + c3 TINYINT(3) NOT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; +INSERT INTO dbz_228_tinyint_unsigned VALUES (default, 255, 255, 127); +INSERT INTO dbz_228_tinyint_unsigned VALUES (default, 155, 155, -100); +INSERT INTO dbz_228_tinyint_unsigned VALUES (default, 0, 0, -128); + + +-- DBZ-228 handle unsigned SMALLINT UNSIGNED +CREATE TABLE dbz_228_smallint_unsigned ( + id int auto_increment NOT NULL, + c1 SMALLINT UNSIGNED ZEROFILL NOT NULL, + c2 SMALLINT UNSIGNED NOT NULL, + c3 SMALLINT NOT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; +INSERT INTO dbz_228_smallint_unsigned VALUES (default, 65535, 65535, 32767); +INSERT INTO dbz_228_smallint_unsigned VALUES (default, 45535, 45535, -12767); +INSERT INTO dbz_228_smallint_unsigned VALUES (default, 0, 0, -32768); + + +-- DBZ-228 handle unsigned MEDIUMINT UNSIGNED +CREATE TABLE dbz_228_mediumint_unsigned ( + id int auto_increment NOT NULL, + c1 MEDIUMINT UNSIGNED ZEROFILL NOT NULL, + c2 MEDIUMINT UNSIGNED NOT NULL, + c3 MEDIUMINT NOT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; +INSERT INTO dbz_228_mediumint_unsigned VALUES (default, 16777215, 16777215, 8388607); +INSERT INTO dbz_228_mediumint_unsigned VALUES (default, 10777215, 10777215, -6388607); +INSERT INTO dbz_228_mediumint_unsigned VALUES (default, 0, 0, -8388608); + +-- DBZ-228 handle unsigned INT UNSIGNED +CREATE TABLE dbz_228_int_unsigned ( + id int auto_increment NOT NULL, + c1 int(11) UNSIGNED ZEROFILL NOT NULL, + c2 int(11) UNSIGNED NOT NULL, + c3 int(11) NOT NULL, + c4 int(5) UNSIGNED ZEROFILL NOT NULL, + c5 int(5) UNSIGNED NOT NULL , + c6 int(5) NOT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; +INSERT INTO dbz_228_int_unsigned VALUES (default, 4294967295, 4294967295, 2147483647, 4294967295, 4294967295, 2147483647); +INSERT INTO dbz_228_int_unsigned VALUES (default, 3294967295, 3294967295, -1147483647, 3294967295, 3294967295, -1147483647); +INSERT INTO dbz_228_int_unsigned VALUES (default, 0, 0, -2147483648, 0, 0, -2147483648); + + +-- DBZ-228 handle unsigned BIGINT UNSIGNED +CREATE TABLE dbz_228_bigint_unsigned ( + id int auto_increment NOT NULL, + c1 BIGINT UNSIGNED ZEROFILL NOT NULL, + c2 BIGINT UNSIGNED NOT NULL, + c3 BIGINT NOT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; +INSERT INTO dbz_228_bigint_unsigned VALUES (default, 18446744073709551615, 18446744073709551615, 9223372036854775807); +INSERT INTO dbz_228_bigint_unsigned VALUES (default, 14446744073709551615, 14446744073709551615, -1223372036854775807); +INSERT INTO dbz_228_bigint_unsigned VALUES (default, 0, 0, -9223372036854775808); + +-- DBZ-1185 handle SERIAL type alias +CREATE TABLE dbz_1185_serial ( + id SERIAL NOT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; +INSERT INTO dbz_1185_serial VALUES (10); +INSERT INTO dbz_1185_serial VALUES (default); +INSERT INTO dbz_1185_serial VALUES (18446744073709551615); + +-- DBZ-1185 handle SERIAL default value +CREATE TABLE dbz_1185_serial_default_value ( + id SMALLINT UNSIGNED SERIAL DEFAULT VALUE NOT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; +INSERT INTO dbz_1185_serial_default_value VALUES (10); +INSERT INTO dbz_1185_serial_default_value VALUES (default); +INSERT INTO dbz_1185_serial_default_value VALUES (1000); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/year_test.sql b/tests/integration_tests/debezium/sql/debezium/year_test.sql new file mode 100644 index 00000000000..ddadd0f97b5 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/year_test.sql @@ -0,0 +1,48 @@ +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: year_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE dbz_1143_year_test ( + id INT AUTO_INCREMENT NOT NULL, + y18 YEAR, + y0018 YEAR, + y2018 YEAR, + d18 DATE, + d0018 DATE, + d2018 DATE, + dt18 DATETIME, + dt0018 DATETIME, + dt2018 DATETIME, + y78 YEAR, + y0078 YEAR, + y1978 YEAR, + d78 DATE, + d0078 DATE, + d1978 DATE, + dt78 DATETIME, + dt0078 DATETIME, + dt1978 DATETIME, + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO dbz_1143_year_test VALUES ( + default, + '18', + '0018', + '2018', + '18-04-01', + '0018-04-01', + '2018-04-01', + '18-04-01 12:34:56', + '0018-04-01 12:34:56', + '2018-04-01 12:34:56', + '78', + '0078', + '1978', + '78-04-01', + '0078-04-01', + '1978-04-01', + '78-04-01 12:34:56', + '0078-04-01 12:34:56', + '1978-04-01 12:34:56' +); diff --git a/tests/integration_tests/debezium/sql/dml.sql b/tests/integration_tests/debezium/sql/dml.sql new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration_tests/debezium/src/db_helper.go b/tests/integration_tests/debezium/src/db_helper.go new file mode 100644 index 00000000000..25b1e3ece80 --- /dev/null +++ b/tests/integration_tests/debezium/src/db_helper.go @@ -0,0 +1,67 @@ +package main + +import ( + "database/sql" + "strings" + + "go.uber.org/zap" +) + +type DBHelper struct { + db *sql.DB + kind Kind +} + +func NewDBHelper(kind Kind) *DBHelper { + return &DBHelper{ + db: nil, + kind: kind, + } +} + +func (h *DBHelper) MustOpen(connStringPattern string, dbName string) { + connString := strings.Replace(connStringPattern, "{db}", dbName, -1) + db, err := sql.Open("mysql", connString) + if err != nil { + logger.Panic( + "Failed to open db", + zap.String("kind", string(h.kind)), + zap.String("conn", connString), + zap.Error(err)) + } + err = db.Ping() + if err != nil { + logger.Panic( + "Failed to open db", + zap.String("kind", string(h.kind)), + zap.String("conn", connString), + zap.Error(err)) + } + h.db = db +} + +func (h *DBHelper) MustExec(query string) { + _, err := h.db.Exec(query) + if err != nil { + logger.Panic( + "Failed to execute query", + zap.String("kind", string(h.kind)), + zap.String("query", query), + zap.Error(err)) + } +} + +func (h *DBHelper) Exec(query string) error { + _, err := h.db.Exec(query) + return err +} + +func (h *DBHelper) MustClose() { + err := h.db.Close() + if err != nil { + logger.Panic( + "Failed to close connection", + zap.String("kind", string(h.kind)), + zap.Error(err)) + } +} diff --git a/tests/integration_tests/debezium/src/logger.go b/tests/integration_tests/debezium/src/logger.go new file mode 100644 index 00000000000..f899e1b6cf7 --- /dev/null +++ b/tests/integration_tests/debezium/src/logger.go @@ -0,0 +1,12 @@ +package main + +import ( + prettyconsole "github.com/thessem/zap-prettyconsole" + "go.uber.org/zap" +) + +var logger *zap.Logger + +func init() { + logger = prettyconsole.NewLogger(zap.DebugLevel) +} diff --git a/tests/integration_tests/debezium/src/main.go b/tests/integration_tests/debezium/src/main.go new file mode 100644 index 00000000000..74ab5a205eb --- /dev/null +++ b/tests/integration_tests/debezium/src/main.go @@ -0,0 +1,92 @@ +package main + +import ( + "flag" + "fmt" + "os" + "time" + + _ "github.com/go-sql-driver/mysql" + "github.com/google/uuid" + _ "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/segmentio/kafka-go" + "go.uber.org/zap" +) + +var ( + dbName *string + testCaseDir *string + kafkaAddr *string +) + +var ( + dbMySQL *DBHelper + dbTiDB *DBHelper + + readerDebezium *kafka.Reader + readerTiCDC *kafka.Reader +) + +type Kind string + +const ( + KindMySQL Kind = "mysql" + KindTiDB Kind = "tidb" +) + +func prepareDBConn(kind Kind, connString string) *DBHelper { + db := NewDBHelper(kind) + db.MustOpen(connString, *dbName) + return db +} + +func prepareKafkaConn(topic string) *kafka.Reader { + r := kafka.NewReader(kafka.ReaderConfig{ + Brokers: []string{*kafkaAddr}, + Topic: topic, + MaxBytes: 10e6, // 10MB + RetentionTime: time.Hour, + + // Config below ensures we will not read history messages. + GroupID: uuid.New().String(), + StartOffset: kafka.LastOffset, + }) + return r +} + +func buildDefaultDBConnStr(port int) string { + return fmt.Sprintf("root@tcp(127.0.0.1:%d)/{db}?allowNativePasswords=true", port) +} + +func main() { + dbConnMySQL := flag.String("db.mysql", buildDefaultDBConnStr(3306), "The connection string to connect to a MySQL instance") + dbConnTiDB := flag.String("db.tidb", buildDefaultDBConnStr(4000), "The connection string to connect to a TiDB instance") + kafkaAddr = flag.String("cdc.kafka", "127.0.0.1:9094", "") + topicDebezium := flag.String("cdc.topic.debezium", "output_debezium", "") + topicTiCDC := flag.String("cdc.topic.ticdc", "output_ticdc", "") + dbName = flag.String("db", "test", "The database to test with") + testCaseDir = flag.String("dir", "./sql", "The directory of SQL test cases") + + flag.Parse() + + logger.Info("Info", + zap.String("db.mysql", *dbConnMySQL), + zap.String("db.tidb", *dbConnTiDB), + zap.String("cdc.mysql", fmt.Sprintf("kafka://%s/%s", *kafkaAddr, *topicDebezium)), + zap.String("cdc.tidb", fmt.Sprintf("kafka://%s/%s", *kafkaAddr, *topicTiCDC)), + ) + + readerDebezium = prepareKafkaConn(*topicDebezium) + defer readerDebezium.Close() + readerTiCDC = prepareKafkaConn(*topicTiCDC) + defer readerTiCDC.Close() + + dbMySQL = prepareDBConn(KindMySQL, *dbConnMySQL) + defer dbMySQL.MustClose() + dbTiDB = prepareDBConn(KindTiDB, *dbConnTiDB) + defer dbTiDB.MustClose() + + if !runAllTestCases(*testCaseDir) { + os.Exit(1) + } +} diff --git a/tests/integration_tests/debezium/src/test_cases.go b/tests/integration_tests/debezium/src/test_cases.go new file mode 100644 index 00000000000..3e1de3fc643 --- /dev/null +++ b/tests/integration_tests/debezium/src/test_cases.go @@ -0,0 +1,219 @@ +package main + +import ( + "context" + "encoding/json" + "fmt" + "os" + "path/filepath" + "strings" + "time" + + "github.com/alecthomas/chroma/quick" + "github.com/fatih/color" + "github.com/google/go-cmp/cmp" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/segmentio/kafka-go" + "go.uber.org/zap" +) + +var ( + nFailed = 0 + nPassed = 0 +) + +func parseSQLText(data string) (res []ast.StmtNode, warns []error, err error) { + p := parser.New() + statements, warns, err := p.Parse(data, "utf8mb4", "utf8mb4_bin") + return statements, warns, err +} + +func readAndParseSQLText(sqlFilePath string) []ast.StmtNode { + data, err := os.ReadFile(sqlFilePath) + if err != nil { + logger.Panic("Failed to read test case file", + zap.String("case", sqlFilePath), + zap.Error(err)) + } + statements, warns, err := parseSQLText(string(data)) + if warns != nil { + logger.Warn("Meet warnings when parsing SQL", + zap.String("case", sqlFilePath), + zap.Any("warnings", warns)) + } + if err != nil { + logger.Panic("Failed to parse SQL", + zap.String("case", sqlFilePath), + zap.Error(err)) + } + return statements +} + +func runAllTestCases(dir string) bool { + var files []string + err := filepath.Walk(dir, func(path string, info os.FileInfo, err error) error { + if info.IsDir() { + return nil + } + if !strings.HasSuffix(info.Name(), ".sql") { + return nil + } + files = append(files, path) + return nil + }) + if err != nil { + logger.Panic("Failed to read test case directory", zap.String("dir", dir), zap.Error(err)) + } + + for _, path := range files { + logger.Info("Run", zap.String("case", path)) + runTestCase(path) + } + + if nFailed > 0 { + logger.Error( + "Test finished with error", + zap.Int("passed", nPassed), + zap.Int("failed", nFailed)) + } else { + logger.Info( + "All tests pass", + zap.Int("passed", nPassed), + zap.Int("failed", nFailed)) + } + + return nFailed == 0 +} + +var ( + statementKindsToWaitCDCRecord = map[string]bool{ + "Delete": true, + "Insert": true, + "Replace": true, + "Update": true, + } +) + +func resetDB(db *DBHelper) { + db.MustExec("drop database if exists `" + *dbName + "`;") + db.MustExec("create database `" + *dbName + "`;") + db.MustExec("use `" + *dbName + "`;") +} + +func runTestCase(testCasePath string) bool { + resetDB(dbMySQL) + resetDB(dbTiDB) + + hasError := false + stmtAsts := readAndParseSQLText(testCasePath) + for _, stmt := range stmtAsts { + query := strings.TrimSpace(stmt.Text()) + statementKind := ast.GetStmtLabel(stmt) + waitCDC := false + if v, ok := statementKindsToWaitCDCRecord[statementKind]; ok { + waitCDC = v + } + if runSingleQuery(query, waitCDC) { + nPassed++ + } else { + nFailed++ + hasError = true + } + } + return hasError +} + +func fetchNextCDCRecord(reader *kafka.Reader, kind Kind) (map[string]any, string, error) { + for { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + m, err := reader.FetchMessage(ctx) + if err != nil { + cancel() + return nil, "", fmt.Errorf("Failed to read CDC record of %s: %w", kind, err) + } + cancel() + + var obj map[string]any + err = json.Unmarshal(m.Value, &obj) + if err != nil { + return nil, "", fmt.Errorf("Failed to parse CDC record of %s (msg=%s): %w", kind, m.Value, err) + } + + // Ignore DDL events in the Debezium's output + if kind == KindMySQL { + schema, ok := obj["schema"] + if !ok { + return nil, "", fmt.Errorf("Unexpected CDC record of %s: schema field not exist in %s", kind, m.Value) + } + if schema.(map[string]any)["name"] == "io.debezium.connector.mysql.SchemaChangeValue" { + continue + } + } + + return obj, string(m.Value), nil + } +} + +var ( + ignoredRecordPaths = map[string]bool{ + `{map[string]any}["schema"]`: true, + `{map[string]any}["payload"].(map[string]any)["source"]`: true, + `{map[string]any}["payload"].(map[string]any)["ts_ms"]`: true, + } +) + +var ( + headingColor = color.New(color.FgHiWhite, color.Bold) +) + +func runSingleQuery(query string, waitCDC bool) bool { + dbMySQL.MustExec(query) + dbTiDB.MustExec(query) + + if !waitCDC { + return true + } + + onError := func(err error) { + fmt.Println("==========================================") + logger.Error("Test failed", zap.Error(err)) + headingColor.Print("\nSQL:\n\n") + quick.Highlight(os.Stdout, query, "sql", "terminal16m", "vs") + fmt.Println() + } + + objDebezium, _, err := fetchNextCDCRecord(readerDebezium, KindMySQL) + if err != nil { + onError(err) + return false + } + // logger.Info("Debezium output", zap.Any("record", objDebezium)) + + objTiCDC, _, err := fetchNextCDCRecord(readerTiCDC, KindTiDB) + if err != nil { + onError(err) + return false + } + // logger.Info("TiCDC output", zap.Any("record", objTiCDC)) + + cmpOption := cmp.FilterPath( + func(p cmp.Path) bool { + path := p.GoString() + _, shouldIgnore := ignoredRecordPaths[path] + return shouldIgnore + }, + cmp.Ignore(), + ) + + if diff := cmp.Diff(objDebezium, objTiCDC, cmpOption); diff != "" { + err = fmt.Errorf("Found mismatch CDC record") + onError(err) + headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") + quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") + fmt.Println() + return false + } + + return true +} From efb519c0bc664051a72b139236d67c46c458c144 Mon Sep 17 00:00:00 2001 From: Wish Date: Thu, 30 Nov 2023 15:20:01 +0800 Subject: [PATCH 02/25] Add unit tests Signed-off-by: Wish --- pkg/sink/codec/debezium/codec.go | 75 ++-- pkg/sink/codec/debezium/codec_test.go | 346 ++++++++++++++++++ pkg/sink/codec/debezium/encoder.go | 11 +- .../debezium/docker-compose.yml | 27 +- .../connector_read_binary_field_test.sql | 18 - .../debezium/sql/debezium/connector_test.sql | 6 +- .../debezium/default_value_all_zero_time.sql | 31 -- ...rated.sql => default_value_generated.sqlx} | 0 .../debezium/sql/debezium/json_test.sqlx | 137 ------- .../debezium/sql/debezium/regression_test.sql | 48 --- .../source_type_as_schema_parameter_test.sql | 16 - .../debezium/sql/debezium/tinyint_test.sql | 15 - .../debezium/sql/debezium/year_test.sql | 48 --- tests/integration_tests/debezium/sql/dml.sql | 24 ++ tests/integration_tests/debezium/src/main.go | 2 +- .../debezium/src/test_cases.go | 169 ++++++--- 16 files changed, 574 insertions(+), 399 deletions(-) create mode 100644 pkg/sink/codec/debezium/codec_test.go delete mode 100644 tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql rename tests/integration_tests/debezium/sql/debezium/{default_value_generated.sql => default_value_generated.sqlx} (100%) delete mode 100644 tests/integration_tests/debezium/sql/debezium/json_test.sqlx delete mode 100644 tests/integration_tests/debezium/sql/debezium/source_type_as_schema_parameter_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/year_test.sql diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index 39fd68952ab..7df08ec2cf4 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -1,3 +1,16 @@ +// Copyright 2024 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 debezium import ( @@ -6,7 +19,6 @@ import ( "encoding/json" "fmt" "strconv" - "strings" "time" "github.com/pingcap/tidb/parser/mysql" @@ -74,7 +86,9 @@ type debeziumMsgSource struct { } type Codec struct { - config *common.Config + config *common.Config + clusterID string + nowFunc func() time.Time } // See https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-data-types @@ -178,18 +192,16 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( col.Name)) case mysql.TypeDate, mysql.TypeNewDate: if v, ok := col.Value.(string); ok { - if v == "0000-00-00" { + t, err := time.Parse("2006-01-02", v) + if err != nil { + // For example, time may be invalid like 1000-00-00 + // return nil, nil if mysql.HasNotNullFlag(ft.GetFlag()) { return 0, nil } else { return nil, nil } } - t, err := time.Parse("2006-01-02", v) - if err != nil { - // For example, time may be invalid like 1000-00-00 - return nil, nil - } return t.Unix() / 60 / 60 / 24, nil } return nil, cerror.WrapError( @@ -205,19 +217,15 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( // TODO: For Default Value = CURRENT_TIMESTAMP, the result is incorrect. if v, ok := col.Value.(string); ok { - if strings.HasPrefix(v, "0000-00-00") { + t, err := time.Parse("2006-01-02 15:04:05.999999", v) + if err != nil { + // For example, time may be 1000-00-00 if mysql.HasNotNullFlag(ft.GetFlag()) { return 0, nil } else { return nil, nil } } - - t, err := time.Parse("2006-01-02 15:04:05.999999", v) - if err != nil { - // For example, time may be 1000-00-00 - return nil, nil - } if ft.GetDecimal() <= 3 { return t.UnixMilli(), nil } else { @@ -241,14 +249,14 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( // > the server by default. If this fails, it must be specified explicitly by the database // > connectionTimeZone MySQL configuration option. if v, ok := col.Value.(string); ok { - // In cdc/entry/codec.go/unflatten, TIMESTAMP types are formatted - // in a timezone comes from config.GetGlobalServerConfig().TZ - // So here we must parse it in the same time zone, - // and then output in UTC as Debezium requires. t, err := time.ParseInLocation("2006-01-02 15:04:05.999999", v, c.config.TimeZone) if err != nil { // For example, time may be invalid like 1000-00-00 - return nil, nil + if mysql.HasNotNullFlag(ft.GetFlag()) { + t = time.Unix(0, 0) + } else { + return nil, nil + } } str := t.UTC().Format("2006-01-02T15:04:05") @@ -287,9 +295,26 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( "unexpected column value type %T for time column %s", col.Value, col.Name)) - default: - return col.Value, nil + case mysql.TypeLonglong: + if col.Flag.IsUnsigned() { + // Handle with BIGINT UNSIGNED. + // Debezium always produce INT64 instead of UINT64 for BIGINT. + if v, ok := col.Value.(uint64); ok { + return int64(v), nil + } + return nil, cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + errors.Errorf( + "unexpected column value type %T for unsigned bigint column %s", + col.Value, + col.Name)) + } + + // Note: Although Debezium's doc claims to use INT32 for INT, but it + // actually uses INT64. Debezium also uses INT32 for SMALLINT. + // So we only handle with TypeLonglong here. } + return col.Value, nil } func (c *Codec) rowChangeToDebeziumMsg(e *model.RowChangedEvent) (*debeziumDataChangeMsg, error) { @@ -298,7 +323,7 @@ func (c *Codec) rowChangeToDebeziumMsg(e *model.RowChangedEvent) (*debeziumDataC source := &debeziumMsgSource{ Version: "2.4.0.Final", Connector: "TiCDC", - Name: "", // TODO + Name: c.clusterID, TsMs: commitTime.UnixMilli(), Snapshot: false, Db: e.Table.Schema, @@ -312,12 +337,12 @@ func (c *Codec) rowChangeToDebeziumMsg(e *model.RowChangedEvent) (*debeziumDataC Query: nil, CommitTs: e.CommitTs, - ClusterID: "", // TODO + ClusterID: c.clusterID, } payload := &debeziumDataChangeMsgPayload{ Source: source, - TsMs: time.Now().UnixMilli(), + TsMs: c.nowFunc().UnixMilli(), Transaction: nil, } diff --git a/pkg/sink/codec/debezium/codec_test.go b/pkg/sink/codec/debezium/codec_test.go new file mode 100644 index 00000000000..87d3733ce0a --- /dev/null +++ b/pkg/sink/codec/debezium/codec_test.go @@ -0,0 +1,346 @@ +// Copyright 2024 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 debezium + +import ( + "math" + "testing" + "time" + + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/rowcodec" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/stretchr/testify/require" +) + +func TestEncodeInsert(t *testing.T) { + codec := &Codec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test-cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + + e := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "table1"}, + Columns: []*model.Column{{ + Name: "tiny", + Value: int64(1), Type: mysql.TypeTiny, + }}, + ColInfos: []rowcodec.ColInfo{{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiny), + }}, + } + + data, err := codec.EncodeRowChangedEvent(e) + require.Nil(t, err) + require.JSONEq(t, `{ + "payload": { + "before": null, + "after": { + "tiny": 1 + }, + "op": "c", + "source": { + "cluster_id": "test-cluster", + "name": "test-cluster", + "commit_ts": 1, + "connector": "TiCDC", + "db": "test", + "table": "table1", + "ts_ms": 0, + "file": "", + "gtid": null, + "pos": 0, + "query": null, + "row": 0, + "server_id": 0, + "snapshot": false, + "thread": 0, + "version": "2.4.0.Final" + }, + "ts_ms": 1701326309000, + "transaction": null + } + }`, string(data)) +} + +func TestEncodeUpdate(t *testing.T) { + codec := &Codec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test-cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + + e := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "table1"}, + Columns: []*model.Column{{ + Name: "tiny", + Value: int64(1), Type: mysql.TypeTiny, + }}, + PreColumns: []*model.Column{{ + Name: "tiny", + Value: int64(2), Type: mysql.TypeTiny, + }}, + ColInfos: []rowcodec.ColInfo{{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiny), + }}, + } + + data, err := codec.EncodeRowChangedEvent(e) + require.Nil(t, err) + require.JSONEq(t, `{ + "payload": { + "before": { + "tiny": 2 + }, + "after": { + "tiny": 1 + }, + "op": "u", + "source": { + "cluster_id": "test-cluster", + "name": "test-cluster", + "commit_ts": 1, + "connector": "TiCDC", + "db": "test", + "table": "table1", + "ts_ms": 0, + "file": "", + "gtid": null, + "pos": 0, + "query": null, + "row": 0, + "server_id": 0, + "snapshot": false, + "thread": 0, + "version": "2.4.0.Final" + }, + "ts_ms": 1701326309000, + "transaction": null + } + }`, string(data)) +} + +func TestEncodeDelete(t *testing.T) { + codec := &Codec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test-cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + + e := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "table1"}, + PreColumns: []*model.Column{{ + Name: "tiny", + Value: int64(2), Type: mysql.TypeTiny, + }}, + ColInfos: []rowcodec.ColInfo{{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiny), + }}, + } + + data, err := codec.EncodeRowChangedEvent(e) + require.Nil(t, err) + require.JSONEq(t, `{ + "payload": { + "before": { + "tiny": 2 + }, + "after": null, + "op": "d", + "source": { + "cluster_id": "test-cluster", + "name": "test-cluster", + "commit_ts": 1, + "connector": "TiCDC", + "db": "test", + "table": "table1", + "ts_ms": 0, + "file": "", + "gtid": null, + "pos": 0, + "query": null, + "row": 0, + "server_id": 0, + "snapshot": false, + "thread": 0, + "version": "2.4.0.Final" + }, + "ts_ms": 1701326309000, + "transaction": null + } + }`, string(data)) +} + +func TestEncodeDataTypes(t *testing.T) { + codec := &Codec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test-cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.TimeZone, _ = time.LoadLocation("Asia/Shanghai") + + e := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "table1"}, + Columns: []*model.Column{{ + Name: "tiny", + Value: int64(1), Type: mysql.TypeTiny, + }, { + Name: "varchar", + Value: []byte("foo"), Type: mysql.TypeVarchar, + }, { + Name: "varchar_bin", + Value: []byte("foo"), Type: mysql.TypeVarchar, Flag: model.BinaryFlag, + }, { + Name: "bit_1_1", + Value: uint64(1), Type: mysql.TypeBit, + }, { + Name: "bit_1_0", + Value: uint64(0), Type: mysql.TypeBit, + }, { + Name: "bit_4", + Value: uint64(13), Type: mysql.TypeBit, + }, { + Name: "decimal", + Value: "129012.1230000", Type: mysql.TypeNewDecimal, + }, { + Name: "bigint_unsigned", + Value: uint64(math.MaxUint64), Type: mysql.TypeLonglong, Flag: model.UnsignedFlag, + }, { + Name: "time", + Value: "12:34:56.1234", Type: mysql.TypeDuration, + }, { + Name: "timestamp", + // Note: This is the human readable time in the Asia/Shanghai timezone. + Value: "2023-11-30 07:10:08.245", Type: mysql.TypeTimestamp, + }, { + Name: "date", + Value: "2023-11-30", Type: mysql.TypeDate, + }}, + ColInfos: []rowcodec.ColInfo{{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiny), + }, { + ID: 2, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarchar), + }, { + ID: 3, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarchar), + }, { + ID: 4, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeBit).SetFlen(1).BuildP(), + }, { + ID: 5, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeBit).SetFlen(1).BuildP(), + }, { + ID: 6, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeBit).SetFlen(4).BuildP(), + }, { + ID: 7, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeNewDecimal), + }, { + ID: 8, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).SetFlag(mysql.UnsignedFlag).BuildP(), + }, { + ID: 9, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeDuration).SetDecimal(4).BuildP(), + }, { + ID: 10, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeTimestamp).SetDecimal(3).BuildP(), + }, { + ID: 11, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDate), + }}, + } + + data, err := codec.EncodeRowChangedEvent(e) + require.Nil(t, err) + require.JSONEq(t, `{ + "payload": { + "before": null, + "after": { + "tiny": 1, + "varchar": "foo", + "varchar_bin": "Zm9v", + "bit_1_0": false, + "bit_1_1": true, + "bit_4": "DQ==", + "decimal": 129012.123, + "bigint_unsigned": -1, + "time": 45296123400, + "timestamp": "2023-11-29T23:10:08.245Z", + "date": 19691 + }, + "op": "c", + "source": { + "cluster_id": "test-cluster", + "name": "test-cluster", + "commit_ts": 1, + "connector": "TiCDC", + "db": "test", + "table": "table1", + "ts_ms": 0, + "file": "", + "gtid": null, + "pos": 0, + "query": null, + "row": 0, + "server_id": 0, + "snapshot": false, + "thread": 0, + "version": "2.4.0.Final" + }, + "ts_ms": 1701326309000, + "transaction": null + } + }`, string(data)) +} diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go index 3d4a45edd8a..002665d9c71 100644 --- a/pkg/sink/codec/debezium/encoder.go +++ b/pkg/sink/codec/debezium/encoder.go @@ -1,4 +1,4 @@ -// Copyright 2023 PingCAP, Inc. +// Copyright 2024 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ import ( "bytes" "context" "encoding/binary" + "time" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" @@ -100,14 +101,16 @@ func (d *BatchEncoder) reset() { } // newBatchEncoder creates a new Debezium BatchEncoder. -func newBatchEncoder(config *common.Config) codec.RowEventEncoder { +func newBatchEncoder(c *common.Config) codec.RowEventEncoder { batch := &BatchEncoder{ keyBuf: &bytes.Buffer{}, valueBuf: &bytes.Buffer{}, callbackBuf: make([]func(), 0), - config: config, + config: c, codec: &Codec{ - config: config, + config: c, + clusterID: config.GetGlobalServerConfig().ClusterID, + nowFunc: time.Now, }, } batch.reset() diff --git a/tests/integration_tests/debezium/docker-compose.yml b/tests/integration_tests/debezium/docker-compose.yml index b6cf14c9153..5aa0b16384a 100644 --- a/tests/integration_tests/debezium/docker-compose.yml +++ b/tests/integration_tests/debezium/docker-compose.yml @@ -45,12 +45,21 @@ services: - CONFIG_STORAGE_TOPIC=my_connect_configs - OFFSET_STORAGE_TOPIC=my_connect_offsets - STATUS_STORAGE_TOPIC=my_connect_statuses - # watcher: # For Debug Purpose - # restart: always - # image: quay.io/debezium/kafka:2.4 - # depends_on: - # - kafka - # command: watch-topic -a -k output_debezium - # environment: - # - ZOOKEEPER_CONNECT=zookeeper:2181 - # - KAFKA_BROKER=kafka:9092 + watcher_dbz: # For Debug Purpose + restart: always + image: quay.io/debezium/kafka:2.4 + depends_on: + - kafka + command: watch-topic -a -k output_debezium + environment: + - ZOOKEEPER_CONNECT=zookeeper:2181 + - KAFKA_BROKER=kafka:9092 + watcher_ticdc: # For Debug Purpose + restart: always + image: quay.io/debezium/kafka:2.4 + depends_on: + - kafka + command: watch-topic -a -k output_ticdc + environment: + - ZOOKEEPER_CONNECT=zookeeper:2181 + - KAFKA_BROKER=kafka:9092 diff --git a/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql b/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql deleted file mode 100644 index 0878fd3ec34..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql +++ /dev/null @@ -1,18 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: connector_read_binary_field_test --- ---------------------------------------------------------------------------------------------------------------- - --- Create a table, mainly MySQL time type fields -CREATE TABLE binary_field -( - id INT AUTO_INCREMENT PRIMARY KEY, - now_time TIME, - now_date DATE, - now_date_time DATETIME, - now_time_stamp TIMESTAMP -) ENGINE = innodb - AUTO_INCREMENT = 1 - DEFAULT CHARSET = utf8; - -INSERT INTO binary_field -VALUES (default, now(), now(), now(), now()); diff --git a/tests/integration_tests/debezium/sql/debezium/connector_test.sql b/tests/integration_tests/debezium/sql/debezium/connector_test.sql index d6534176e00..8ae03647b01 100644 --- a/tests/integration_tests/debezium/sql/debezium/connector_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/connector_test.sql @@ -11,7 +11,7 @@ CREATE TABLE products ( ); ALTER TABLE products AUTO_INCREMENT = 101; -INSERT INTO products +INSERT INTO products VALUES (default,"scooter","Small 2-wheel scooter",3.14), (default,"car battery","12V car battery",8.1), (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), @@ -65,11 +65,9 @@ CREATE TABLE orders ( FOREIGN KEY ordered_product (product_id) REFERENCES products(id) ) AUTO_INCREMENT = 10001; -INSERT INTO orders +INSERT INTO orders VALUES (default, '2016-01-16', 1001, 1, 102), (default, '2016-01-17', 1002, 2, 105), (default, '2016-02-18', 1004, 3, 109), (default, '2016-02-19', 1002, 2, 106), (default, '16-02-21', 1003, 1, 107); - -CREATE DATABASE IF NOT EXISTS emptydb; diff --git a/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql b/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql deleted file mode 100644 index 5b6320725a5..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql +++ /dev/null @@ -1,31 +0,0 @@ -CREATE TABLE all_zero_date_and_time_table ( - A TIMESTAMP NOT NULL DEFAULT '0000-00-00 00:00:00', - B TIMESTAMP NULL DEFAULT '0000-00-00 00:00:00', - C TIMESTAMP DEFAULT '0000-00-00 00:00:00' NOT NULL, - D TIMESTAMP DEFAULT '0000-00-00 00:00:00' NULL, - E DATE NOT NULL DEFAULT '0000-00-00', - F DATE NULL DEFAULT '0000-00-00', - G DATE DEFAULT '0000-00-00' NOT NULL, - H DATE DEFAULT '0000-00-00' NULL, - I DATETIME NOT NULL DEFAULT '0000-00-00 00:00:00', - J DATETIME NULL DEFAULT '0000-00-00 00:00:00', - K DATETIME DEFAULT '0000-00-00 00:00:00' NOT NULL, - L DATETIME DEFAULT '0000-00-00 00:00:00' NULL, - PK INT PRIMARY KEY -); -INSERT INTO all_zero_date_and_time_table -VALUES (DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, 1 ); - -CREATE TABLE part_zero_date_and_time_table ( - A TIMESTAMP NOT NULL DEFAULT '0000-00-00 00:00:00', - B TIMESTAMP NULL DEFAULT '0000-00-00 00:00:00', - C DATETIME NOT NULL DEFAULT '0000-00-00 01:00:00.000', - D DATETIME NULL DEFAULT '0000-00-00 01:00:00.000', - E DATE NOT NULL DEFAULT '1000-00-00 01:00:00.000', - F DATE NULL DEFAULT '1000-00-00 01:00:00.000', - G TIME DEFAULT '0000-00-00 01:00:00.000' NOT NULL, - H TIME DEFAULT '0000-00-00 01:00:00.000' NULL, - PK INT PRIMARY KEY -); -INSERT INTO part_zero_date_and_time_table -VALUES (DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, 1 ); diff --git a/tests/integration_tests/debezium/sql/debezium/default_value_generated.sql b/tests/integration_tests/debezium/sql/debezium/default_value_generated.sqlx similarity index 100% rename from tests/integration_tests/debezium/sql/debezium/default_value_generated.sql rename to tests/integration_tests/debezium/sql/debezium/default_value_generated.sqlx diff --git a/tests/integration_tests/debezium/sql/debezium/json_test.sqlx b/tests/integration_tests/debezium/sql/debezium/json_test.sqlx deleted file mode 100644 index 336dd038928..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/json_test.sqlx +++ /dev/null @@ -1,137 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: json_test --- ---------------------------------------------------------------------------------------------------------------- --- The integration test for this database expects to scan all of the binlog events associated with this database --- without error or problems. The integration test does not modify any records in this database, so this script --- must contain all operations to these tables. --- --- This relies upon MySQL 5.7's JSON datatype. - --- DBZ-126 handle JSON column types ... -CREATE TABLE dbz_126_jsontable ( - id INT AUTO_INCREMENT NOT NULL, - json JSON, - expectedJdbcStr VARCHAR(256), -- value that we get back from JDBC - expectedBinlogStr VARCHAR(256), -- value we parse from the binlog - PRIMARY KEY(id) -) DEFAULT CHARSET=utf8; -INSERT INTO dbz_126_jsontable VALUES (default,NULL, - NULL, - NULL); -INSERT INTO dbz_126_jsontable VALUES (default,'{"a": 2}', - '{"a": 2}', - '{"a":2}'); -INSERT INTO dbz_126_jsontable VALUES (default,'[1, 2]', - '[1, 2]', - '[1,2]'); -INSERT INTO dbz_126_jsontable VALUES (default,'{"key1": "value1", "key2": "value2"}', - '{"key1": "value1", "key2": "value2"}', - '{"key1":"value1","key2":"value2"}'); -INSERT INTO dbz_126_jsontable VALUES (default,'["a", "b",1]', - '["a", "b",1]', - '["a","b",1]'); -INSERT INTO dbz_126_jsontable VALUES (default,'{"k1": "v1", "k2": {"k21": "v21", "k22": "v22"}, "k3": ["a", "b", 1]}', - '{"k1": "v1", "k2": {"k21": "v21", "k22": "v22"}, "k3": ["a", "b", 1]}', - '{"k1":"v1","k2":{"k21":"v21","k22":"v22"},"k3":["a","b",1]}'); -INSERT INTO dbz_126_jsontable VALUES (default,'{"a": "b", "c": "d", "ab": "abc", "bc": ["x", "y"]}', - '{"a": "b", "c": "d", "ab": "abc", "bc": ["x", "y"]}', - '{"a":"b","c":"d","ab":"abc","bc":["x","y"]}'); -INSERT INTO dbz_126_jsontable VALUES (default,'["here", ["I", "am"], "!!!"]', - '["here", ["I", "am"], "!!!"]', - '["here",["I","am"],"!!!"]'); -INSERT INTO dbz_126_jsontable VALUES (default,'"scalar string"', - '"scalar string"', - '"scalar string"'); -INSERT INTO dbz_126_jsontable VALUES (default,'true', - 'true', - 'true'); -INSERT INTO dbz_126_jsontable VALUES (default,'false', - 'false', - 'false'); -INSERT INTO dbz_126_jsontable VALUES (default,'null', - 'null', - 'null'); -INSERT INTO dbz_126_jsontable VALUES (default,'-1', - '-1', - '-1'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST(1 AS UNSIGNED) AS JSON), - '1', - '1'); -INSERT INTO dbz_126_jsontable VALUES (default,'32767', - '32767', - '32767'); -INSERT INTO dbz_126_jsontable VALUES (default,'32768', - '32768', - '32768'); -INSERT INTO dbz_126_jsontable VALUES (default,'-32768', - '-32768', - '-32768'); -INSERT INTO dbz_126_jsontable VALUES (default,'2147483647', -- INT32 - '2147483647', - '2147483647'); -INSERT INTO dbz_126_jsontable VALUES (default,'2147483648', -- INT64 - '2147483648', - '2147483648'); -INSERT INTO dbz_126_jsontable VALUES (default,'-2147483648', -- INT32 - '-2147483648', - '-2147483648'); -INSERT INTO dbz_126_jsontable VALUES (default,'-2147483649', -- INT64 - '-2147483649', - '-2147483649'); -INSERT INTO dbz_126_jsontable VALUES (default,'18446744073709551615', -- INT64 - '18446744073709551615', - '18446744073709551615'); -INSERT INTO dbz_126_jsontable VALUES (default,'18446744073709551616', -- BigInteger - '18446744073709551616', - '18446744073709551616'); -INSERT INTO dbz_126_jsontable VALUES (default,'3.14', - '3.14', - '3.14'); -INSERT INTO dbz_126_jsontable VALUES (default,'{}', - '{}', - '{}'); -INSERT INTO dbz_126_jsontable VALUES (default,'[]', - '[]', - '[]'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('2015-01-15 23:24:25' AS DATETIME) AS JSON), - '"2015-01-15 23:24:25"', - '"2015-01-15 23:24:25"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('2015-01-15 23:24:25.12' AS DATETIME(3)) AS JSON), - '"2015-01-15 23:24:25.12"', - '"2015-01-15 23:24:25.12"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('2015-01-15 23:24:25.0237' AS DATETIME(3)) AS JSON), - '"2015-01-15 23:24:25.024"', - '"2015-01-15 23:24:25.024"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('23:24:25' AS TIME) AS JSON), - '"23:24:25"', - '"23:24:25"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('23:24:25.12' AS TIME(3)) AS JSON), - '"23:24:25.12"', - '"23:24:25.12"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('23:24:25.0237' AS TIME(3)) AS JSON), - '"23:24:25.024"', - '"23:24:25.024"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(CAST('2015-01-15' AS DATE) AS JSON), - '"2015-01-15"', - '"2015-01-15"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(TIMESTAMP'2015-01-15 23:24:25' AS JSON), - '"2015-01-15 23:24:25"', - '"2015-01-15 23:24:25"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(TIMESTAMP'2015-01-15 23:24:25.12' AS JSON), - '"2015-01-15 23:24:25.12"', - '"2015-01-15 23:24:25.12"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(TIMESTAMP'2015-01-15 23:24:25.0237' AS JSON), - '"2015-01-15 23:24:25.0237"', - '"2015-01-15 23:24:25.0237"'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(UNIX_TIMESTAMP('2015-01-15 23:24:25') AS JSON), - '1421364265', - '1421364265'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST('[]' AS CHAR CHARACTER SET 'ascii'), - '[]', - '[]'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(x'cafe' AS JSON), -- BLOB as Base64 - '"yv4="', - '"yv4="'); -INSERT INTO dbz_126_jsontable VALUES (default,CAST(x'cafebabe' AS JSON), -- BLOB as Base64 - '"yv66vg=="', - '"yv66vg=="'); diff --git a/tests/integration_tests/debezium/sql/debezium/regression_test.sql b/tests/integration_tests/debezium/sql/debezium/regression_test.sql index 42f07803169..5d396f4d44d 100644 --- a/tests/integration_tests/debezium/sql/debezium/regression_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/regression_test.sql @@ -74,33 +74,6 @@ CREATE TABLE dbz_102_charsettest ( ) ENGINE=InnoDB AUTO_INCREMENT=2001 DEFAULT CHARSET=utf8; INSERT INTO dbz_102_charsettest VALUES (default, "产品"); --- DBZ-114 handle zero-value dates -CREATE TABLE dbz_114_zerovaluetest ( - c1 DATE, - c2 TIME(2), - c3 DATETIME(2), - c4 TIMESTAMP(2), - PK INT PRIMARY KEY -); -INSERT IGNORE INTO dbz_114_zerovaluetest VALUES ('0000-00-00', '00:00:00.000', '0000-00-00 00:00:00.000', '0000-00-00 00:00:00.000', 1); -INSERT IGNORE INTO dbz_114_zerovaluetest VALUES ('0001-00-00', '00:01:00.000', '0001-00-00 00:00:00.000', '0001-00-00 00:00:00.000', 2); - --- DBZ-1318 handle zero-value dates in when zero dates allowed -CREATE TABLE dbz_1318_zerovaluetest ( - c1 DATE, - c2 TIME(2), - c3 DATETIME(2), - c4 TIMESTAMP(2), - nnc1 DATE NOT NULL, - nnc2 TIME(2) NOT NULL, - nnc3 DATETIME(2) NOT NULL, - PK INT PRIMARY KEY -); -set sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION'; -INSERT IGNORE INTO dbz_1318_zerovaluetest VALUES ('0000-00-00', '00:00:00.000', '0000-00-00 00:00:00.000', '0000-00-00 00:00:00.000', '0000-00-00', '00:00:00.000', '0000-00-00 00:00:00.000', 1); -INSERT IGNORE INTO dbz_1318_zerovaluetest VALUES ('0001-00-00', '00:01:00.000', '0000-00-00 14:02:10.000', '0000-00-00 14:02:10.000', '0001-00-00', '00:01:00.000', '0000-00-00 14:02:10.000', 2); -set sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION'; - -- DBZ-123 handle bit values, including bit field literals CREATE TABLE dbz_123_bitvaluetest ( c1 BIT, @@ -127,9 +100,6 @@ VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), (default,"Edward","Walker","ed@walker.com"), (default,"Anne","Kretchmar","annek@noanswer.org"); -CREATE TABLE dbz_104_customers LIKE connector_test.customers; -INSERT INTO dbz_104_customers SELECT * FROM connector_test.customers; - -- DBZ-147 handle decimal value CREATE TABLE dbz_147_decimalvalues ( pk_column int auto_increment NOT NULL, @@ -139,24 +109,6 @@ CREATE TABLE dbz_147_decimalvalues ( INSERT INTO dbz_147_decimalvalues (pk_column, decimal_value) VALUES(default, 12345.67); -/* --- DBZ-162 handle function declarations with newline characters -CREATE FUNCTION fnDbz162( p_creditLimit DOUBLE ) RETURNS VARCHAR(10) - DETERMINISTIC -BEGIN - DECLARE lvl VARCHAR(10)$$ - IF p_creditLimit > 50000 THEN - SET lvl = 'PLATINUM'$$ - ELSEIF (p_creditLimit <= 50000 AND p_creditLimit >= 10000) THEN - SET lvl = 'GOLD'$$ - ELSEIF p_creditLimit < 10000 THEN - SET lvl = 'SILVER'$$ - END IF$$ - RETURN (lvl)$$ -END$$ -; -*/ - -- DBZ-195 handle numeric values CREATE TABLE dbz_195_numvalues ( id int auto_increment NOT NULL, diff --git a/tests/integration_tests/debezium/sql/debezium/source_type_as_schema_parameter_test.sql b/tests/integration_tests/debezium/sql/debezium/source_type_as_schema_parameter_test.sql deleted file mode 100644 index d4bd690360c..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/source_type_as_schema_parameter_test.sql +++ /dev/null @@ -1,16 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: source_type_as_schema_parameter_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE dbz_644_source_type_mapped_as_schema_parameter_test ( - id INT AUTO_INCREMENT NOT NULL, - c1 INT, - c2 MEDIUMINT, - c3a NUMERIC(5,2), - c3b VARCHAR(128), - f1 FLOAT(10), - f2 FLOAT(8,4), - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8; - -INSERT INTO dbz_644_source_type_mapped_as_schema_parameter_test VALUES (default, 123, 456, 789.01, 'test', 1.228, 234.56); diff --git a/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql b/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql index 76fd26f47f2..341dcc8f25b 100644 --- a/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql @@ -1,16 +1,3 @@ -CREATE TABLE `DBZ1773` ( - id int not null primary key auto_increment, - ti TINYINT SIGNED, - ti1 TINYINT(1) SIGNED, - ti2 TINYINT(2), - b BOOLEAN -) ENGINE=InnoDB AUTO_INCREMENT=10851 DEFAULT CHARSET=utf8; - -CREATE TABLE `DBZ2085` ( - id int not null primary key auto_increment, - b BOOLEAN NOT NULL DEFAULT 0 -) ENGINE=InnoDB AUTO_INCREMENT=10851 DEFAULT CHARSET=utf8; - CREATE TABLE `DBZ5236` ( id int(11) not null primary key auto_increment, ti1 tinyint(3) unsigned NOT NULL DEFAULT '0', @@ -18,6 +5,4 @@ CREATE TABLE `DBZ5236` ( ti3 tinyint(1) NOT NULL DEFAULT '1' ) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; -INSERT INTO DBZ1773 VALUES (DEFAULT, 100, 5, 50, TRUE); -INSERT INTO DBZ2085 VALUES (DEFAULT, 1); INSERT INTO DBZ5236 VALUES (DEFAULT, 1, 1, 0); diff --git a/tests/integration_tests/debezium/sql/debezium/year_test.sql b/tests/integration_tests/debezium/sql/debezium/year_test.sql deleted file mode 100644 index ddadd0f97b5..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/year_test.sql +++ /dev/null @@ -1,48 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: year_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE dbz_1143_year_test ( - id INT AUTO_INCREMENT NOT NULL, - y18 YEAR, - y0018 YEAR, - y2018 YEAR, - d18 DATE, - d0018 DATE, - d2018 DATE, - dt18 DATETIME, - dt0018 DATETIME, - dt2018 DATETIME, - y78 YEAR, - y0078 YEAR, - y1978 YEAR, - d78 DATE, - d0078 DATE, - d1978 DATE, - dt78 DATETIME, - dt0078 DATETIME, - dt1978 DATETIME, - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8; - -INSERT INTO dbz_1143_year_test VALUES ( - default, - '18', - '0018', - '2018', - '18-04-01', - '0018-04-01', - '2018-04-01', - '18-04-01 12:34:56', - '0018-04-01 12:34:56', - '2018-04-01 12:34:56', - '78', - '0078', - '1978', - '78-04-01', - '0078-04-01', - '1978-04-01', - '78-04-01 12:34:56', - '0078-04-01 12:34:56', - '1978-04-01 12:34:56' -); diff --git a/tests/integration_tests/debezium/sql/dml.sql b/tests/integration_tests/debezium/sql/dml.sql index e69de29bb2d..f8fc3f5644c 100644 --- a/tests/integration_tests/debezium/sql/dml.sql +++ b/tests/integration_tests/debezium/sql/dml.sql @@ -0,0 +1,24 @@ +CREATE TABLE foo( + PK INT PRIMARY KEY, + COL INT +); + +INSERT INTO foo VALUES (1, 1); + +INSERT INTO foo VALUES (2, 2); + +INSERT INTO foo VALUES (3, 3); + +/* Update PK */ +UPDATE foo SET PK = 5, COL = 5 WHERE COL = 3; + +/* Update Multiple Rows */ +UPDATE foo SET COL = 4; + +/* Update Single Row */ +UPDATE foo SET COL = 1 WHERE PK = 5; + +/* Update No Rows */ +UPDATE foo SET COL = 1 WHERE PK = 100; + +DELETE FROM foo WHERE PK = 3; diff --git a/tests/integration_tests/debezium/src/main.go b/tests/integration_tests/debezium/src/main.go index 74ab5a205eb..5f16ba5fc7c 100644 --- a/tests/integration_tests/debezium/src/main.go +++ b/tests/integration_tests/debezium/src/main.go @@ -36,7 +36,7 @@ const ( func prepareDBConn(kind Kind, connString string) *DBHelper { db := NewDBHelper(kind) - db.MustOpen(connString, *dbName) + db.MustOpen(connString, "") return db } diff --git a/tests/integration_tests/debezium/src/test_cases.go b/tests/integration_tests/debezium/src/test_cases.go index 3e1de3fc643..248c31d89bc 100644 --- a/tests/integration_tests/debezium/src/test_cases.go +++ b/tests/integration_tests/debezium/src/test_cases.go @@ -3,10 +3,13 @@ package main import ( "context" "encoding/json" + "errors" "fmt" "os" "path/filepath" + "regexp" "strings" + "sync" "time" "github.com/alecthomas/chroma/quick" @@ -86,75 +89,115 @@ func runAllTestCases(dir string) bool { return nFailed == 0 } -var ( - statementKindsToWaitCDCRecord = map[string]bool{ - "Delete": true, - "Insert": true, - "Replace": true, - "Update": true, - } -) - func resetDB(db *DBHelper) { db.MustExec("drop database if exists `" + *dbName + "`;") db.MustExec("create database `" + *dbName + "`;") db.MustExec("use `" + *dbName + "`;") } +var ( + regexpExpectRows = regexp.MustCompile(`TICDC_TEST_EXPECT_ROWS:(\d+)`) +) + func runTestCase(testCasePath string) bool { resetDB(dbMySQL) resetDB(dbTiDB) + statementKindsToWaitCDCRecord := map[string]bool{ + "Delete": true, + "Insert": true, + "Replace": true, + "Update": true, + } + hasError := false stmtAsts := readAndParseSQLText(testCasePath) for _, stmt := range stmtAsts { query := strings.TrimSpace(stmt.Text()) + + waitCDCRows := false statementKind := ast.GetStmtLabel(stmt) - waitCDC := false - if v, ok := statementKindsToWaitCDCRecord[statementKind]; ok { - waitCDC = v + if v, ok := statementKindsToWaitCDCRecord[statementKind]; v && ok { + waitCDCRows = true } - if runSingleQuery(query, waitCDC) { + + if runSingleQuery(query, waitCDCRows) { nPassed++ } else { nFailed++ hasError = true } } + return hasError } -func fetchNextCDCRecord(reader *kafka.Reader, kind Kind) (map[string]any, string, error) { +func fetchNextCDCRecord(reader *kafka.Reader, kind Kind, timeout time.Duration) (map[string]any, error) { for { - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), timeout) m, err := reader.FetchMessage(ctx) + cancel() + if err != nil { - cancel() - return nil, "", fmt.Errorf("Failed to read CDC record of %s: %w", kind, err) + if errors.Is(err, context.DeadlineExceeded) { + return nil, nil + } + return nil, fmt.Errorf("Failed to read CDC record of %s: %w", kind, err) + } + + if len(m.Value) == 0 { + continue } - cancel() var obj map[string]any err = json.Unmarshal(m.Value, &obj) if err != nil { - return nil, "", fmt.Errorf("Failed to parse CDC record of %s (msg=%s): %w", kind, m.Value, err) + return nil, fmt.Errorf("Failed to parse CDC record of %s (msg=%s): %w", kind, m.Value, err) } // Ignore DDL events in the Debezium's output if kind == KindMySQL { schema, ok := obj["schema"] if !ok { - return nil, "", fmt.Errorf("Unexpected CDC record of %s: schema field not exist in %s", kind, m.Value) + return nil, fmt.Errorf("Unexpected CDC record of %s: schema field not exist in %s", kind, m.Value) } if schema.(map[string]any)["name"] == "io.debezium.connector.mysql.SchemaChangeValue" { continue } } - return obj, string(m.Value), nil + return obj, nil } } +func fetchAllCDCRecords(reader *kafka.Reader, kind Kind) []map[string]any { + var records []map[string]any + for { + waitTimeout := time.Millisecond * 500 + if len(records) == 0 { + // Wait a bit longer for the first record + waitTimeout = 5 * time.Second + } + + obj, err := fetchNextCDCRecord(reader, kind, waitTimeout) + if err != nil { + logger.Error( + "Received error when fetching CDC record", + zap.Error(err), + zap.String("kind", string(kind))) + break + } + if obj == nil { + // No more records + break + } + + records = append(records, obj) + } + + return records +} + var ( ignoredRecordPaths = map[string]bool{ `{map[string]any}["schema"]`: true, @@ -167,35 +210,72 @@ var ( headingColor = color.New(color.FgHiWhite, color.Bold) ) -func runSingleQuery(query string, waitCDC bool) bool { - dbMySQL.MustExec(query) - dbTiDB.MustExec(query) +func printObj(obj any) { + v, _ := json.MarshalIndent(obj, "", " ") + quick.Highlight(os.Stdout, string(v), "json", "terminal16m", "vs") + fmt.Println() +} + +func runSingleQuery(query string, waitCDCRows bool) bool { + { + wg := &sync.WaitGroup{} + wg.Add(2) + go func() { + dbMySQL.MustExec(query) + wg.Done() + }() + go func() { + dbTiDB.MustExec(query) + wg.Done() + }() + wg.Wait() + } - if !waitCDC { + if !waitCDCRows { return true } + testCasePassed := true onError := func(err error) { fmt.Println("==========================================") logger.Error("Test failed", zap.Error(err)) headingColor.Print("\nSQL:\n\n") quick.Highlight(os.Stdout, query, "sql", "terminal16m", "vs") fmt.Println() + testCasePassed = false } - objDebezium, _, err := fetchNextCDCRecord(readerDebezium, KindMySQL) - if err != nil { - onError(err) - return false + var objsDebezium []map[string]any + var objsTiCDC []map[string]any + { + wg := &sync.WaitGroup{} + wg.Add(2) + go func() { + objsDebezium = fetchAllCDCRecords(readerDebezium, KindMySQL) + wg.Done() + }() + go func() { + objsTiCDC = fetchAllCDCRecords(readerTiCDC, KindTiDB) + wg.Done() + }() + wg.Wait() } - // logger.Info("Debezium output", zap.Any("record", objDebezium)) - objTiCDC, _, err := fetchNextCDCRecord(readerTiCDC, KindTiDB) - if err != nil { - onError(err) - return false + if len(objsDebezium) != len(objsTiCDC) { + onError(fmt.Errorf( + "Mismatch CDC rows: Got %d rows from Debezium and %d rows from TiCDC", + len(objsDebezium), + len(objsTiCDC))) + headingColor.Print("\nDebezium output:\n\n") + for _, obj := range objsDebezium { + printObj(obj) + } + headingColor.Print("\nTiCDC output:\n\n") + for _, obj := range objsTiCDC { + printObj(obj) + } + return testCasePassed } - // logger.Info("TiCDC output", zap.Any("record", objTiCDC)) cmpOption := cmp.FilterPath( func(p cmp.Path) bool { @@ -206,14 +286,17 @@ func runSingleQuery(query string, waitCDC bool) bool { cmp.Ignore(), ) - if diff := cmp.Diff(objDebezium, objTiCDC, cmpOption); diff != "" { - err = fmt.Errorf("Found mismatch CDC record") - onError(err) - headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") - quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") - fmt.Println() - return false + for i := 0; i < len(objsDebezium); i++ { + objDebezium := objsDebezium[i] + objTiCDC := objsTiCDC[i] + if diff := cmp.Diff(objDebezium, objTiCDC, cmpOption); diff != "" { + onError(fmt.Errorf("Found mismatch CDC record (output row #%d)", i+1)) + headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") + quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") + fmt.Println() + continue + } } - return true + return testCasePassed } From d7c6b5da1390356a79310698b1b0dc5668795444 Mon Sep 17 00:00:00 2001 From: Wish Date: Thu, 30 Nov 2023 16:43:51 +0800 Subject: [PATCH 03/25] Adjust timeout Signed-off-by: Wish --- .../debezium/sql/debezium/default_value_generated.sqlx | 7 ------- tests/integration_tests/debezium/src/test_cases.go | 6 +++++- 2 files changed, 5 insertions(+), 8 deletions(-) delete mode 100644 tests/integration_tests/debezium/sql/debezium/default_value_generated.sqlx diff --git a/tests/integration_tests/debezium/sql/debezium/default_value_generated.sqlx b/tests/integration_tests/debezium/sql/debezium/default_value_generated.sqlx deleted file mode 100644 index 46bca73985a..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/default_value_generated.sqlx +++ /dev/null @@ -1,7 +0,0 @@ -CREATE TABLE GENERATED_TABLE ( - A SMALLINT UNSIGNED, - B SMALLINT UNSIGNED AS (2 * A), - C SMALLINT UNSIGNED AS (3 * A) NOT NULL, - PK INT PRIMARY KEY -); -INSERT INTO GENERATED_TABLE VALUES (15, DEFAULT, DEFAULT, 1); diff --git a/tests/integration_tests/debezium/src/test_cases.go b/tests/integration_tests/debezium/src/test_cases.go index 248c31d89bc..208f1b970bd 100644 --- a/tests/integration_tests/debezium/src/test_cases.go +++ b/tests/integration_tests/debezium/src/test_cases.go @@ -176,7 +176,11 @@ func fetchAllCDCRecords(reader *kafka.Reader, kind Kind) []map[string]any { waitTimeout := time.Millisecond * 500 if len(records) == 0 { // Wait a bit longer for the first record - waitTimeout = 5 * time.Second + if kind == KindMySQL { + waitTimeout = 3 * time.Second + } else if kind == KindTiDB { + waitTimeout = 10 * time.Second + } } obj, err := fetchNextCDCRecord(reader, kind, waitTimeout) From 4147e4542248b6b84c00cdce59cf2918c7fecbc8 Mon Sep 17 00:00:00 2001 From: Wish Date: Fri, 1 Dec 2023 10:40:22 +0800 Subject: [PATCH 04/25] Improve performance and reduce memory usage Signed-off-by: Wish --- go.mod | 3 +- go.sum | 2 + pkg/sink/codec/debezium/codec.go | 255 +++++++++++++------------- pkg/sink/codec/debezium/codec_test.go | 115 +++++++++++- pkg/sink/codec/debezium/encoder.go | 5 +- pkg/util/json_writer.go | 173 +++++++++++++++++ 6 files changed, 417 insertions(+), 136 deletions(-) create mode 100644 pkg/util/json_writer.go diff --git a/go.mod b/go.mod index 5967048505b..ca2e5e6a97f 100644 --- a/go.mod +++ b/go.mod @@ -87,6 +87,7 @@ require ( github.com/swaggo/gin-swagger v1.2.0 github.com/swaggo/swag v1.8.3 github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 + github.com/thanhpk/randstr v1.0.6 github.com/tikv/client-go/v2 v2.0.8-0.20230925032502-44b0cf7aba2b github.com/tikv/pd v1.1.0-beta.0.20230203015356-248b3f0be132 github.com/tikv/pd/client v0.0.0-20230912103610-2f57a9f050eb @@ -255,7 +256,7 @@ require ( github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df // indirect github.com/jonboulle/clockwork v0.3.0 // indirect github.com/josharian/intern v1.0.0 // indirect - github.com/json-iterator/go v1.1.12 // indirect + github.com/json-iterator/go v1.1.12 github.com/klauspost/cpuid v1.3.1 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect diff --git a/go.sum b/go.sum index fd72156f024..470848e6f14 100644 --- a/go.sum +++ b/go.sum @@ -1236,6 +1236,8 @@ github.com/swaggo/swag v1.8.3 h1:3pZSSCQ//gAH88lfmxM3Cd1+JCsxV8Md6f36b9hrZ5s= github.com/swaggo/swag v1.8.3/go.mod h1:jMLeXOOmYyjk8PvHTsXBdrubsNd9gUJTTCzL5iBnseg= github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 h1:xQdMZ1WLrgkkvOZ/LDQxjVxMLdby7osSh4ZEVa5sIjs= github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954/go.mod h1:u2MKkTVTVJWe5D1rCvame8WqhBd88EuIwODJZ1VHCPM= +github.com/thanhpk/randstr v1.0.6 h1:psAOktJFD4vV9NEVb3qkhRSMvYh4ORRaj1+w/hn4B+o= +github.com/thanhpk/randstr v1.0.6/go.mod h1:M/H2P1eNLZzlDwAzpkkkUvoyNNMbzRGhESZuEQk3r0U= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index 7df08ec2cf4..6441df95e35 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -14,20 +14,23 @@ package debezium import ( - "encoding/base64" "encoding/binary" - "encoding/json" "fmt" + "io" "strconv" "time" + _ "github.com/goccy/go-json" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/errors" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/pd/pkg/utils/tsoutil" ) @@ -66,8 +69,8 @@ type debeziumMsgSource struct { Version string `json:"version"` Connector string `json:"connector"` Name string `json:"name"` - // TsMs: In the source object, ts_ms indicates the time that the change was made in the database. - // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events + // // TsMs: In the source object, ts_ms indicates the time that the change was made in the database. + // // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events TsMs int64 `json:"ts_ms"` Snapshot bool `json:"snapshot"` Db string `json:"db"` @@ -91,10 +94,24 @@ type Codec struct { nowFunc func() time.Time } +func (c *Codec) writeColumnsAsField(writer *util.JSONWriter, fieldName string, cols []*model.Column, colInfos []rowcodec.ColInfo) error { + var err error + writer.WriteObjectField(fieldName, func() { + for i, col := range cols { + err = c.writeDebeziumField(writer, col, colInfos[i].Ft) + if err != nil { + break + } + } + }) + return err +} + // See https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-data-types -func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) (any, error) { +func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, ft *types.FieldType) error { if col.Value == nil { - return nil, nil + writer.WriteNullField(col.Name) + return nil } switch col.Type { case mysql.TypeBit: @@ -105,19 +122,20 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( // contain the specified number of bits. n := ft.GetFlen() if n == 1 { - return v != 0, nil + writer.WriteBoolField(col.Name, v != 0) + return nil } else { - buf := make([]byte, 8) - binary.LittleEndian.PutUint64(buf, uint64(v)) - + var buf [8]byte + binary.LittleEndian.PutUint64(buf[:], v) numBytes := n / 8 if n%8 != 0 { numBytes += 1 } - return base64.StdEncoding.EncodeToString(buf[:numBytes]), nil + c.writeBinaryField(writer, col.Name, buf[:numBytes]) + return nil } } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for bit column %s", @@ -127,9 +145,10 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: if col.Flag.IsBinary() { if v, ok := col.Value.([]byte); ok { - return base64.StdEncoding.EncodeToString(v), nil + c.writeBinaryField(writer, col.Name, v) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for binary string column %s", @@ -137,9 +156,10 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( col.Name)) } else { if v, ok := col.Value.([]byte); ok { - return string(v), nil + writer.WriteStringField(col.Name, string(hack.String(v))) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for non-binary string column %s", @@ -150,11 +170,12 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( if v, ok := col.Value.(uint64); ok { enumVar, err := types.ParseEnumValue(ft.GetElems(), v) if err != nil { - return nil, cerror.WrapError(cerror.ErrDebeziumEncodeFailed, err) + return cerror.WrapError(cerror.ErrDebeziumEncodeFailed, err) } - return enumVar.Name, nil + writer.WriteStringField(col.Name, enumVar.Name) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for enum column %s", @@ -164,11 +185,12 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( if v, ok := col.Value.(uint64); ok { setVar, err := types.ParseSetValue(ft.GetElems(), v) if err != nil { - return nil, cerror.WrapError(cerror.ErrDebeziumEncodeFailed, err) + return cerror.WrapError(cerror.ErrDebeziumEncodeFailed, err) } - return setVar.Name, nil + writer.WriteStringField(col.Name, setVar.Name) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for set column %s", @@ -178,13 +200,14 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( if v, ok := col.Value.(string); ok { floatV, err := strconv.ParseFloat(v, 64) if err != nil { - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, err) } - return floatV, nil + writer.WriteFloat64Field(col.Name, floatV) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for decimal column %s", @@ -197,14 +220,17 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( // For example, time may be invalid like 1000-00-00 // return nil, nil if mysql.HasNotNullFlag(ft.GetFlag()) { - return 0, nil + writer.WriteInt64Field(col.Name, 0) + return nil } else { - return nil, nil + writer.WriteNullField(col.Name) + return nil } } - return t.Unix() / 60 / 60 / 24, nil + writer.WriteInt64Field(col.Name, t.Unix()/60/60/24) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for date column %s", @@ -221,18 +247,22 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( if err != nil { // For example, time may be 1000-00-00 if mysql.HasNotNullFlag(ft.GetFlag()) { - return 0, nil + writer.WriteInt64Field(col.Name, 0) + return nil } else { - return nil, nil + writer.WriteNullField(col.Name) + return nil } } if ft.GetDecimal() <= 3 { - return t.UnixMilli(), nil + writer.WriteInt64Field(col.Name, t.UnixMilli()) + return nil } else { - return t.UnixMicro(), nil + writer.WriteInt64Field(col.Name, t.UnixMicro()) + return nil } } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for datetime column %s", @@ -255,7 +285,8 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( if mysql.HasNotNullFlag(ft.GetFlag()) { t = time.Unix(0, 0) } else { - return nil, nil + writer.WriteNullField(col.Name) + return nil } } @@ -267,9 +298,10 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( } str += "Z" - return str, nil + writer.WriteStringField(col.Name, str) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for timestamp column %s", @@ -283,13 +315,15 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( ctx := &stmtctx.StatementContext{} d, _, _, err := types.StrToDuration(ctx, v, ft.GetDecimal()) if err != nil { - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, err) } - return d.Microseconds(), nil + + writer.WriteInt64Field(col.Name, d.Microseconds()) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for time column %s", @@ -300,9 +334,10 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( // Handle with BIGINT UNSIGNED. // Debezium always produce INT64 instead of UINT64 for BIGINT. if v, ok := col.Value.(uint64); ok { - return int64(v), nil + writer.WriteInt64Field(col.Name, int64(v)) + return nil } - return nil, cerror.WrapError( + return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, errors.Errorf( "unexpected column value type %T for unsigned bigint column %s", @@ -314,95 +349,67 @@ func (c *Codec) convertToDebeziumField(col *model.Column, ft *types.FieldType) ( // actually uses INT64. Debezium also uses INT32 for SMALLINT. // So we only handle with TypeLonglong here. } - return col.Value, nil + + writer.WriteAnyField(col.Name, col.Value) + return nil } -func (c *Codec) rowChangeToDebeziumMsg(e *model.RowChangedEvent) (*debeziumDataChangeMsg, error) { - commitTime, _ := tsoutil.ParseTS(e.CommitTs) +func (c *Codec) writeBinaryField(writer *util.JSONWriter, fieldName string, value []byte) { + // TODO: Deal with different binary output later. + writer.WriteBase64StringField(fieldName, value) +} - source := &debeziumMsgSource{ - Version: "2.4.0.Final", - Connector: "TiCDC", - Name: c.clusterID, - TsMs: commitTime.UnixMilli(), - Snapshot: false, - Db: e.Table.Schema, - Table: e.Table.Table, - ServerID: 0, - GtID: nil, - File: "", - Pos: 0, - Row: 0, - Thread: 0, - Query: nil, +func (c *Codec) EncodeRowChangedEvent( + e *model.RowChangedEvent, + dest io.Writer, +) error { + jWriter := util.BorrowJSONWriter(dest) + defer util.ReturnJSONWriter(jWriter) - CommitTs: e.CommitTs, - ClusterID: c.clusterID, - } + commitTime, _ := tsoutil.ParseTS(e.CommitTs) - payload := &debeziumDataChangeMsgPayload{ - Source: source, - TsMs: c.nowFunc().UnixMilli(), - Transaction: nil, - } + var err error - applyBefore := func() error { - payload.Before = make(map[string]any) - for i, col := range e.PreColumns { - value, err := c.convertToDebeziumField(col, e.ColInfos[i].Ft) - if err != nil { - return err - } - payload.Before[col.Name] = value - } - return nil - } + jWriter.WriteObject(func() { + jWriter.WriteObjectField("payload", func() { + jWriter.WriteObjectField("source", func() { + jWriter.WriteStringField("version", "2.4.0.Final") + jWriter.WriteStringField("connector", "TiCDC") + jWriter.WriteStringField("name", c.clusterID) + jWriter.WriteInt64Field("ts_ms", commitTime.UnixMilli()) + jWriter.WriteBoolField("snapshot", false) + jWriter.WriteStringField("db", e.Table.Schema) + jWriter.WriteStringField("table", e.Table.Table) + jWriter.WriteInt64Field("server_id", 0) + jWriter.WriteNullField("gtid") + jWriter.WriteStringField("file", "") + jWriter.WriteInt64Field("pos", 0) + jWriter.WriteInt64Field("row", 0) + jWriter.WriteInt64Field("thread", 0) + jWriter.WriteNullField("query") + jWriter.WriteUint64Field("commit_ts", e.CommitTs) + jWriter.WriteStringField("cluster_id", c.clusterID) + }) + jWriter.WriteInt64Field("ts_ms", c.nowFunc().UnixMilli()) + jWriter.WriteNullField("transaction") - applyAfter := func() error { - payload.After = make(map[string]any) - for i, col := range e.Columns { - value, err := c.convertToDebeziumField(col, e.ColInfos[i].Ft) - if err != nil { - return err + if e.IsInsert() { + jWriter.WriteStringField("op", "c") + jWriter.WriteNullField("before") + err = c.writeColumnsAsField(jWriter, "after", e.Columns, e.ColInfos) + } else if e.IsDelete() { + jWriter.WriteStringField("op", "d") + jWriter.WriteNullField("after") + err = c.writeColumnsAsField(jWriter, "before", e.PreColumns, e.ColInfos) + } else if e.IsUpdate() { + jWriter.WriteStringField("op", "u") + err = c.writeColumnsAsField(jWriter, "before", e.PreColumns, e.ColInfos) + if err == nil { + err = c.writeColumnsAsField(jWriter, "after", e.Columns, e.ColInfos) + } } - payload.After[col.Name] = value - } - return nil - } + }) + }) - if e.IsInsert() { - payload.Op = "c" - payload.Before = nil - if err := applyAfter(); err != nil { - return nil, err - } - } else if e.IsDelete() { - payload.Op = "d" - payload.After = nil - if err := applyBefore(); err != nil { - return nil, err - } - } else if e.IsUpdate() { - payload.Op = "u" - if err := applyBefore(); err != nil { - return nil, err - } - if err := applyAfter(); err != nil { - return nil, err - } - } - - return &debeziumDataChangeMsg{ - Payload: payload, - }, nil -} - -func (c *Codec) EncodeRowChangedEvent( - e *model.RowChangedEvent, -) ([]byte, error) { - m, err := c.rowChangeToDebeziumMsg(e) - if err != nil { - return nil, errors.Trace(err) - } - return json.Marshal(m) + return err } diff --git a/pkg/sink/codec/debezium/codec_test.go b/pkg/sink/codec/debezium/codec_test.go index 87d3733ce0a..e69bee5b4c9 100644 --- a/pkg/sink/codec/debezium/codec_test.go +++ b/pkg/sink/codec/debezium/codec_test.go @@ -14,6 +14,7 @@ package debezium import ( + "bytes" "math" "testing" "time" @@ -25,6 +26,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/stretchr/testify/require" + "github.com/thanhpk/randstr" ) func TestEncodeInsert(t *testing.T) { @@ -49,7 +51,8 @@ func TestEncodeInsert(t *testing.T) { }}, } - data, err := codec.EncodeRowChangedEvent(e) + buf := bytes.NewBuffer(nil) + err := codec.EncodeRowChangedEvent(e, buf) require.Nil(t, err) require.JSONEq(t, `{ "payload": { @@ -79,7 +82,7 @@ func TestEncodeInsert(t *testing.T) { "ts_ms": 1701326309000, "transaction": null } - }`, string(data)) + }`, string(buf.Bytes())) } func TestEncodeUpdate(t *testing.T) { @@ -108,7 +111,8 @@ func TestEncodeUpdate(t *testing.T) { }}, } - data, err := codec.EncodeRowChangedEvent(e) + buf := bytes.NewBuffer(nil) + err := codec.EncodeRowChangedEvent(e, buf) require.Nil(t, err) require.JSONEq(t, `{ "payload": { @@ -140,7 +144,7 @@ func TestEncodeUpdate(t *testing.T) { "ts_ms": 1701326309000, "transaction": null } - }`, string(data)) + }`, string(buf.Bytes())) } func TestEncodeDelete(t *testing.T) { @@ -165,7 +169,8 @@ func TestEncodeDelete(t *testing.T) { }}, } - data, err := codec.EncodeRowChangedEvent(e) + buf := bytes.NewBuffer(nil) + err := codec.EncodeRowChangedEvent(e, buf) require.Nil(t, err) require.JSONEq(t, `{ "payload": { @@ -195,7 +200,7 @@ func TestEncodeDelete(t *testing.T) { "ts_ms": 1701326309000, "transaction": null } - }`, string(data)) + }`, string(buf.Bytes())) } func TestEncodeDataTypes(t *testing.T) { @@ -302,7 +307,8 @@ func TestEncodeDataTypes(t *testing.T) { }}, } - data, err := codec.EncodeRowChangedEvent(e) + buf := bytes.NewBuffer(nil) + err := codec.EncodeRowChangedEvent(e, buf) require.Nil(t, err) require.JSONEq(t, `{ "payload": { @@ -342,5 +348,98 @@ func TestEncodeDataTypes(t *testing.T) { "ts_ms": 1701326309000, "transaction": null } - }`, string(data)) + }`, string(buf.Bytes())) +} + +func BenchmarkEncodeOneTinyColumn(b *testing.B) { + codec := &Codec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test-cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + + e := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "table1"}, + Columns: []*model.Column{{ + Name: "tiny", + Value: int64(10), Type: mysql.TypeTiny, + }}, + ColInfos: []rowcodec.ColInfo{{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiny), + }}, + } + + buf := bytes.NewBuffer(nil) + + b.ResetTimer() + for n := 0; n < b.N; n++ { + buf.Reset() + codec.EncodeRowChangedEvent(e, buf) + } +} + +func BenchmarkEncodeLargeText(b *testing.B) { + codec := &Codec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test-cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + + e := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "table1"}, + Columns: []*model.Column{{ + Name: "str", + Value: []byte(randstr.String(1024)), Type: mysql.TypeVarchar, + }}, + ColInfos: []rowcodec.ColInfo{{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarchar), + }}, + } + + buf := bytes.NewBuffer(nil) + + b.ResetTimer() + for n := 0; n < b.N; n++ { + buf.Reset() + codec.EncodeRowChangedEvent(e, buf) + } +} + +func BenchmarkEncodeLargeBinary(b *testing.B) { + codec := &Codec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test-cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + + e := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "table1"}, + Columns: []*model.Column{{ + Name: "bin", + Value: []byte(randstr.String(1024)), Type: mysql.TypeVarchar, Flag: model.BinaryFlag, + }}, + ColInfos: []rowcodec.ColInfo{{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarchar), + }}, + } + + buf := bytes.NewBuffer(nil) + + b.ResetTimer() + for n := 0; n < b.N; n++ { + buf.Reset() + codec.EncodeRowChangedEvent(e, buf) + } } diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go index 002665d9c71..0fd94e28710 100644 --- a/pkg/sink/codec/debezium/encoder.go +++ b/pkg/sink/codec/debezium/encoder.go @@ -49,11 +49,10 @@ func (d *BatchEncoder) AppendRowChangedEvent( e *model.RowChangedEvent, callback func(), ) error { - value, err := d.codec.EncodeRowChangedEvent(e) + err := d.codec.EncodeRowChangedEvent(e, d.valueBuf) if err != nil { return errors.Trace(err) } - d.valueBuf.Write(value) d.batchSize++ if callback != nil { d.callbackBuf = append(d.callbackBuf, callback) @@ -128,7 +127,7 @@ func NewBatchEncoderBuilder(config *common.Config) codec.RowEventEncoderBuilder } } -// Build a `maxwellBatchEncoder` +// Build a `BatchEncoder` func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { return newBatchEncoder(b.config) } diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go new file mode 100644 index 00000000000..fc4df844cd1 --- /dev/null +++ b/pkg/util/json_writer.go @@ -0,0 +1,173 @@ +// Copyright 2024 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 util + +import ( + "encoding/base64" + "io" + "sync" + + jsoniter "github.com/json-iterator/go" +) + +var jsonAPI = jsoniter.Config{ + EscapeHTML: false, + SortMapKeys: false, +}.Froze() + +var jWriterPool = sync.Pool{ + New: func() interface{} { + return &JSONWriter{} + }, +} + +type JSONWriter struct { + out io.Writer + needPrependComma bool + rawStream *jsoniter.Stream +} + +func BorrowJSONWriter(out io.Writer) *JSONWriter { + w := jWriterPool.Get().(*JSONWriter) + w.rawStream = jsonAPI.BorrowStream(out) + w.needPrependComma = false + w.out = out + return w +} + +func ReturnJSONWriter(w *JSONWriter) { + w.rawStream.Flush() + w.rawStream = nil + w.out = nil + jWriterPool.Put(w) +} + +func (w *JSONWriter) WriteBase64String(b []byte) { + // As we write to out directly so we need to flush the jsoniter stream first. + w.rawStream.Flush() + w.out.Write([]byte(`"`)) + encoder := base64.NewEncoder(base64.StdEncoding, w.out) + _, _ = encoder.Write(b) + encoder.Close() + w.out.Write([]byte(`"`)) +} + +func (w *JSONWriter) WriteObject(objectFieldsWriteFn func()) { + lastNeedPrependComma := w.needPrependComma + w.needPrependComma = false + w.rawStream.WriteObjectStart() + objectFieldsWriteFn() + w.rawStream.WriteObjectEnd() + w.needPrependComma = lastNeedPrependComma +} + +func (w *JSONWriter) WriteBoolField(fieldName string, value bool) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.rawStream.WriteBool(value) +} + +func (w *JSONWriter) WriteIntField(fieldName string, value int) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.rawStream.WriteInt(value) +} + +func (w *JSONWriter) WriteInt64Field(fieldName string, value int64) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.rawStream.WriteInt64(value) +} + +func (w *JSONWriter) WriteUint64Field(fieldName string, value uint64) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.rawStream.WriteUint64(value) +} + +func (w *JSONWriter) WriteFloat64Field(fieldName string, value float64) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.rawStream.WriteFloat64(value) +} + +func (w *JSONWriter) WriteStringField(fieldName string, value string) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.rawStream.WriteString(value) +} + +func (w *JSONWriter) WriteBase64StringField(fieldName string, b []byte) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.WriteBase64String(b) +} + +func (w *JSONWriter) WriteAnyField(fieldName string, value any) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.rawStream.WriteVal(value) +} + +func (w *JSONWriter) WriteObjectField(fieldName string, objectFieldsWriteFn func()) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.WriteObject(objectFieldsWriteFn) +} + +func (w *JSONWriter) WriteNullField(fieldName string) { + if w.needPrependComma { + w.rawStream.WriteMore() + } else { + w.needPrependComma = true + } + w.rawStream.WriteObjectField(fieldName) + w.rawStream.WriteNil() +} From fe6b5679420eaeb49bb8e65f91c5c2f70368b1dd Mon Sep 17 00:00:00 2001 From: Wish Date: Fri, 1 Dec 2023 11:33:03 +0800 Subject: [PATCH 05/25] Remove unused structs Signed-off-by: Wish --- pkg/sink/codec/debezium/codec.go | 77 ++++++++++---------------------- 1 file changed, 23 insertions(+), 54 deletions(-) diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index 6441df95e35..fde371a493f 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -34,60 +34,6 @@ import ( "github.com/tikv/pd/pkg/utils/tsoutil" ) -type debeziumDataChangeMsg struct { - // schema is unsupported - Payload *debeziumDataChangeMsgPayload `json:"payload"` -} - -type debeziumDataChangeMsgPayload struct { - // Before: An optional field that specifies the state of the row before the event occurred. - // When the op field is c for create, the before field is null since this change event is for new content. - // In a delete event value, the before field contains the values that were in the row before - // it was deleted with the database commit. - Before map[string]any `json:"before"` - // After: An optional field that specifies the state of the row after the event occurred. - // Optional field that specifies the state of the row after the event occurred. - // In a delete event value, the after field is null, signifying that the row no longer exists. - After map[string]any `json:"after"` - Source *debeziumMsgSource `json:"source"` - // Op: Mandatory string that describes the type of operation that caused the connector to generate the event. - // Valid values are: - // c = create - // u = update - // d = delete - // r = read (applies to only snapshots) - // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events - Op string `json:"op"` - // TsMs: displays the time at which the connector processed the event - // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events - TsMs int64 `json:"ts_ms"` - // Transaction: Always null - Transaction *struct{} `json:"transaction"` -} - -type debeziumMsgSource struct { - Version string `json:"version"` - Connector string `json:"connector"` - Name string `json:"name"` - // // TsMs: In the source object, ts_ms indicates the time that the change was made in the database. - // // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events - TsMs int64 `json:"ts_ms"` - Snapshot bool `json:"snapshot"` - Db string `json:"db"` - Table string `json:"table"` - ServerID int64 `json:"server_id"` - GtID *string `json:"gtid"` - File string `json:"file"` - Pos int64 `json:"pos"` - Row int32 `json:"row"` - Thread int64 `json:"thread"` - Query *string `json:"query"` - - // The followings are TiDB extended fields - CommitTs uint64 `json:"commit_ts"` - ClusterID string `json:"cluster_id"` -} - type Codec struct { config *common.Config clusterID string @@ -376,6 +322,8 @@ func (c *Codec) EncodeRowChangedEvent( jWriter.WriteStringField("version", "2.4.0.Final") jWriter.WriteStringField("connector", "TiCDC") jWriter.WriteStringField("name", c.clusterID) + // ts_ms: In the source object, ts_ms indicates the time that the change was made in the database. + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events jWriter.WriteInt64Field("ts_ms", commitTime.UnixMilli()) jWriter.WriteBoolField("snapshot", false) jWriter.WriteStringField("db", e.Table.Schema) @@ -387,15 +335,36 @@ func (c *Codec) EncodeRowChangedEvent( jWriter.WriteInt64Field("row", 0) jWriter.WriteInt64Field("thread", 0) jWriter.WriteNullField("query") + + // The followings are TiDB extended fields jWriter.WriteUint64Field("commit_ts", e.CommitTs) jWriter.WriteStringField("cluster_id", c.clusterID) }) + + // ts_ms: displays the time at which the connector processed the event + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events jWriter.WriteInt64Field("ts_ms", c.nowFunc().UnixMilli()) jWriter.WriteNullField("transaction") if e.IsInsert() { + // op: Mandatory string that describes the type of operation that caused the connector to generate the event. + // Valid values are: + // c = create + // u = update + // d = delete + // r = read (applies to only snapshots) + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events jWriter.WriteStringField("op", "c") + + // before: An optional field that specifies the state of the row before the event occurred. + // When the op field is c for create, the before field is null since this change event is for new content. + // In a delete event value, the before field contains the values that were in the row before + // it was deleted with the database commit. jWriter.WriteNullField("before") + + // after: An optional field that specifies the state of the row after the event occurred. + // Optional field that specifies the state of the row after the event occurred. + // In a delete event value, the after field is null, signifying that the row no longer exists. err = c.writeColumnsAsField(jWriter, "after", e.Columns, e.ColInfos) } else if e.IsDelete() { jWriter.WriteStringField("op", "d") From 614499289d6568f5fdb1a670893234bb7439967e Mon Sep 17 00:00:00 2001 From: Wish Date: Fri, 1 Dec 2023 11:39:01 +0800 Subject: [PATCH 06/25] Update naming Signed-off-by: Wish --- pkg/util/json_writer.go | 75 +++++++++++++++++++++-------------------- 1 file changed, 38 insertions(+), 37 deletions(-) diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go index fc4df844cd1..a1882d53fbd 100644 --- a/pkg/util/json_writer.go +++ b/pkg/util/json_writer.go @@ -33,29 +33,30 @@ var jWriterPool = sync.Pool{ } type JSONWriter struct { - out io.Writer + out io.Writer + stream *jsoniter.Stream // `stream` is created over `out` + needPrependComma bool - rawStream *jsoniter.Stream } func BorrowJSONWriter(out io.Writer) *JSONWriter { w := jWriterPool.Get().(*JSONWriter) - w.rawStream = jsonAPI.BorrowStream(out) - w.needPrependComma = false w.out = out + w.stream = jsonAPI.BorrowStream(out) + w.needPrependComma = false return w } func ReturnJSONWriter(w *JSONWriter) { - w.rawStream.Flush() - w.rawStream = nil + w.stream.Flush() w.out = nil + w.stream = nil jWriterPool.Put(w) } func (w *JSONWriter) WriteBase64String(b []byte) { // As we write to out directly so we need to flush the jsoniter stream first. - w.rawStream.Flush() + w.stream.Flush() w.out.Write([]byte(`"`)) encoder := base64.NewEncoder(base64.StdEncoding, w.out) _, _ = encoder.Write(b) @@ -66,108 +67,108 @@ func (w *JSONWriter) WriteBase64String(b []byte) { func (w *JSONWriter) WriteObject(objectFieldsWriteFn func()) { lastNeedPrependComma := w.needPrependComma w.needPrependComma = false - w.rawStream.WriteObjectStart() + w.stream.WriteObjectStart() objectFieldsWriteFn() - w.rawStream.WriteObjectEnd() + w.stream.WriteObjectEnd() w.needPrependComma = lastNeedPrependComma } func (w *JSONWriter) WriteBoolField(fieldName string, value bool) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) - w.rawStream.WriteBool(value) + w.stream.WriteObjectField(fieldName) + w.stream.WriteBool(value) } func (w *JSONWriter) WriteIntField(fieldName string, value int) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) - w.rawStream.WriteInt(value) + w.stream.WriteObjectField(fieldName) + w.stream.WriteInt(value) } func (w *JSONWriter) WriteInt64Field(fieldName string, value int64) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) - w.rawStream.WriteInt64(value) + w.stream.WriteObjectField(fieldName) + w.stream.WriteInt64(value) } func (w *JSONWriter) WriteUint64Field(fieldName string, value uint64) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) - w.rawStream.WriteUint64(value) + w.stream.WriteObjectField(fieldName) + w.stream.WriteUint64(value) } func (w *JSONWriter) WriteFloat64Field(fieldName string, value float64) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) - w.rawStream.WriteFloat64(value) + w.stream.WriteObjectField(fieldName) + w.stream.WriteFloat64(value) } func (w *JSONWriter) WriteStringField(fieldName string, value string) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) - w.rawStream.WriteString(value) + w.stream.WriteObjectField(fieldName) + w.stream.WriteString(value) } func (w *JSONWriter) WriteBase64StringField(fieldName string, b []byte) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) + w.stream.WriteObjectField(fieldName) w.WriteBase64String(b) } func (w *JSONWriter) WriteAnyField(fieldName string, value any) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) - w.rawStream.WriteVal(value) + w.stream.WriteObjectField(fieldName) + w.stream.WriteVal(value) } func (w *JSONWriter) WriteObjectField(fieldName string, objectFieldsWriteFn func()) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) + w.stream.WriteObjectField(fieldName) w.WriteObject(objectFieldsWriteFn) } func (w *JSONWriter) WriteNullField(fieldName string) { if w.needPrependComma { - w.rawStream.WriteMore() + w.stream.WriteMore() } else { w.needPrependComma = true } - w.rawStream.WriteObjectField(fieldName) - w.rawStream.WriteNil() + w.stream.WriteObjectField(fieldName) + w.stream.WriteNil() } From b83e85916aa6c2603fa881d55e831895ffe03de0 Mon Sep 17 00:00:00 2001 From: Wish Date: Mon, 4 Dec 2023 14:53:14 +0800 Subject: [PATCH 07/25] Use SQL in test cases Signed-off-by: Wish --- go.mod | 3 +- go.sum | 4 + pkg/sink/codec/debezium/codec.go | 8 +- pkg/sink/codec/debezium/codec_test.go | 149 ----- pkg/sink/codec/debezium/debezium_test.go | 190 ++++++ .../codec/debezium/testdata/datatype.dbz.json | 617 ++++++++++++++++++ .../codec/debezium/testdata/datatype.ddl.sql | 62 ++ .../codec/debezium/testdata/datatype.dml.sql | 67 ++ .../debezium/sql/data_types.sql | 4 + 9 files changed, 952 insertions(+), 152 deletions(-) create mode 100644 pkg/sink/codec/debezium/debezium_test.go create mode 100644 pkg/sink/codec/debezium/testdata/datatype.dbz.json create mode 100644 pkg/sink/codec/debezium/testdata/datatype.ddl.sql create mode 100644 pkg/sink/codec/debezium/testdata/datatype.dml.sql diff --git a/go.mod b/go.mod index ca2e5e6a97f..c23cbfdef85 100644 --- a/go.mod +++ b/go.mod @@ -41,7 +41,7 @@ require ( github.com/golang/mock v1.6.0 github.com/golang/protobuf v1.5.3 github.com/google/btree v1.1.2 - github.com/google/go-cmp v0.5.9 + github.com/google/go-cmp v0.6.0 github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 github.com/google/uuid v1.3.0 github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 @@ -126,6 +126,7 @@ require ( ) require ( + github.com/AlekSi/pointer v1.2.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 // indirect github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v0.5.1 // indirect diff --git a/go.sum b/go.sum index 470848e6f14..565f5c6ad6f 100644 --- a/go.sum +++ b/go.sum @@ -71,6 +71,8 @@ github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 h1:/vQbFIOMb github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4/go.mod h1:hN7oaIRCjzsZ2dE+yG5k+rsdt3qcwykqK6HVGcKwsw4= github.com/99designs/keyring v1.2.1 h1:tYLp1ULvO7i3fI5vE21ReQuj99QFSs7lGm0xWyJo87o= github.com/99designs/keyring v1.2.1/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwRStKOQ5vOA= +github.com/AlekSi/pointer v1.2.0 h1:glcy/gc4h8HnG2Z3ZECSzZ1IX1x2JxRVuDzaJwQE0+w= +github.com/AlekSi/pointer v1.2.0/go.mod h1:gZGfd3dpW4vEc/UlyfKKi1roIqcCgwOIvb0tSNSBle0= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/AthenZ/athenz v1.10.39 h1:mtwHTF/v62ewY2Z5KWhuZgVXftBej1/Tn80zx4DcawY= github.com/AthenZ/athenz v1.10.39/go.mod h1:3Tg8HLsiQZp81BJY58JBeU2BR6B/H4/0MQGfCwhHNEA= @@ -591,6 +593,8 @@ github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8 github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.1.0 h1:Hsa8mG0dQ46ij8Sl2AYJDUv1oA9/d6Vk+3LG99Oe02g= diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index fde371a493f..de77f3605bb 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -116,7 +116,9 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f if v, ok := col.Value.(uint64); ok { enumVar, err := types.ParseEnumValue(ft.GetElems(), v) if err != nil { - return cerror.WrapError(cerror.ErrDebeziumEncodeFailed, err) + // Invalid enum value inserted in non-strict mode. + writer.WriteStringField(col.Name, "") + return nil } writer.WriteStringField(col.Name, enumVar.Name) return nil @@ -131,7 +133,9 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f if v, ok := col.Value.(uint64); ok { setVar, err := types.ParseSetValue(ft.GetElems(), v) if err != nil { - return cerror.WrapError(cerror.ErrDebeziumEncodeFailed, err) + // Invalid enum value inserted in non-strict mode. + writer.WriteStringField(col.Name, "") + return nil } writer.WriteStringField(col.Name, setVar.Name) return nil diff --git a/pkg/sink/codec/debezium/codec_test.go b/pkg/sink/codec/debezium/codec_test.go index e69bee5b4c9..64a2d72fcdf 100644 --- a/pkg/sink/codec/debezium/codec_test.go +++ b/pkg/sink/codec/debezium/codec_test.go @@ -15,7 +15,6 @@ package debezium import ( "bytes" - "math" "testing" "time" @@ -203,154 +202,6 @@ func TestEncodeDelete(t *testing.T) { }`, string(buf.Bytes())) } -func TestEncodeDataTypes(t *testing.T) { - codec := &Codec{ - config: common.NewConfig(config.ProtocolDebezium), - clusterID: "test-cluster", - nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, - } - codec.config.TimeZone, _ = time.LoadLocation("Asia/Shanghai") - - e := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, - Columns: []*model.Column{{ - Name: "tiny", - Value: int64(1), Type: mysql.TypeTiny, - }, { - Name: "varchar", - Value: []byte("foo"), Type: mysql.TypeVarchar, - }, { - Name: "varchar_bin", - Value: []byte("foo"), Type: mysql.TypeVarchar, Flag: model.BinaryFlag, - }, { - Name: "bit_1_1", - Value: uint64(1), Type: mysql.TypeBit, - }, { - Name: "bit_1_0", - Value: uint64(0), Type: mysql.TypeBit, - }, { - Name: "bit_4", - Value: uint64(13), Type: mysql.TypeBit, - }, { - Name: "decimal", - Value: "129012.1230000", Type: mysql.TypeNewDecimal, - }, { - Name: "bigint_unsigned", - Value: uint64(math.MaxUint64), Type: mysql.TypeLonglong, Flag: model.UnsignedFlag, - }, { - Name: "time", - Value: "12:34:56.1234", Type: mysql.TypeDuration, - }, { - Name: "timestamp", - // Note: This is the human readable time in the Asia/Shanghai timezone. - Value: "2023-11-30 07:10:08.245", Type: mysql.TypeTimestamp, - }, { - Name: "date", - Value: "2023-11-30", Type: mysql.TypeDate, - }}, - ColInfos: []rowcodec.ColInfo{{ - ID: 1, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeTiny), - }, { - ID: 2, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeVarchar), - }, { - ID: 3, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeVarchar), - }, { - ID: 4, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeBit).SetFlen(1).BuildP(), - }, { - ID: 5, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeBit).SetFlen(1).BuildP(), - }, { - ID: 6, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeBit).SetFlen(4).BuildP(), - }, { - ID: 7, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeNewDecimal), - }, { - ID: 8, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).SetFlag(mysql.UnsignedFlag).BuildP(), - }, { - ID: 9, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeDuration).SetDecimal(4).BuildP(), - }, { - ID: 10, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldTypeBuilder().SetType(mysql.TypeTimestamp).SetDecimal(3).BuildP(), - }, { - ID: 11, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeDate), - }}, - } - - buf := bytes.NewBuffer(nil) - err := codec.EncodeRowChangedEvent(e, buf) - require.Nil(t, err) - require.JSONEq(t, `{ - "payload": { - "before": null, - "after": { - "tiny": 1, - "varchar": "foo", - "varchar_bin": "Zm9v", - "bit_1_0": false, - "bit_1_1": true, - "bit_4": "DQ==", - "decimal": 129012.123, - "bigint_unsigned": -1, - "time": 45296123400, - "timestamp": "2023-11-29T23:10:08.245Z", - "date": 19691 - }, - "op": "c", - "source": { - "cluster_id": "test-cluster", - "name": "test-cluster", - "commit_ts": 1, - "connector": "TiCDC", - "db": "test", - "table": "table1", - "ts_ms": 0, - "file": "", - "gtid": null, - "pos": 0, - "query": null, - "row": 0, - "server_id": 0, - "snapshot": false, - "thread": 0, - "version": "2.4.0.Final" - }, - "ts_ms": 1701326309000, - "transaction": null - } - }`, string(buf.Bytes())) -} - func BenchmarkEncodeOneTinyColumn(b *testing.B) { codec := &Codec{ config: common.NewConfig(config.ProtocolDebezium), diff --git a/pkg/sink/codec/debezium/debezium_test.go b/pkg/sink/codec/debezium/debezium_test.go new file mode 100644 index 00000000000..bebec1d1312 --- /dev/null +++ b/pkg/sink/codec/debezium/debezium_test.go @@ -0,0 +1,190 @@ +// Copyright 2024 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. + +//go:build intest +// +build intest + +package debezium + +import ( + "context" + "encoding/json" + "os" + "testing" + "time" + + "github.com/google/go-cmp/cmp" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/spanz" + "github.com/pingcap/tiflow/pkg/util" + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" +) + +type SQLTestHelper struct { + t *testing.T + + helper *entry.SchemaTestHelper + mounter entry.Mounter + + ts uint64 + tableID int64 +} + +func NewSQLTestHelper(t *testing.T, tableName, initialCreateTableDDL string) *SQLTestHelper { + helper := entry.NewSchemaTestHelper(t) + helper.Tk().MustExec("set @@tidb_enable_clustered_index=1;") + helper.Tk().MustExec("use test;") + + changefeed := model.DefaultChangeFeedID("") + + ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) + require.NoError(t, err) + + cfg := config.GetDefaultReplicaConfig() + + filter, err := filter.NewFilter(cfg, "") + require.NoError(t, err) + + schemaStorage, err := entry.NewSchemaStorage(helper.GetCurrentMeta(), + ver.Ver, false, changefeed, util.RoleTester, filter) + require.NoError(t, err) + + job := helper.DDL2Job(initialCreateTableDDL) + err = schemaStorage.HandleDDLJob(job) + require.NoError(t, err) + + ts := schemaStorage.GetLastSnapshot().CurrentTs() + schemaStorage.AdvanceResolvedTs(ver.Ver) + + mounter := entry.NewMounter(schemaStorage, changefeed, time.UTC, filter, cfg.Integrity) + + tableInfo, ok := schemaStorage.GetLastSnapshot().TableByName("test", tableName) + require.True(t, ok) + + return &SQLTestHelper{ + t: t, + helper: helper, + mounter: mounter, + ts: ts, + tableID: tableInfo.ID, + } +} + +func (h *SQLTestHelper) Close() { + h.helper.Close() +} + +func (h *SQLTestHelper) MustExec(query string, args ...interface{}) { + h.helper.Tk().MustExec(query, args...) +} + +func (h *SQLTestHelper) ScanTable() []*model.RowChangedEvent { + txn, err := h.helper.Storage().Begin() + require.Nil(h.t, err) + defer txn.Rollback() //nolint:errcheck + startKey, endKey := spanz.GetTableRange(h.tableID) + kvIter, err := txn.Iter(startKey, endKey) + require.Nil(h.t, err) + defer kvIter.Close() + + ret := make([]*model.RowChangedEvent, 0) + + for kvIter.Valid() { + rawKV := &model.RawKVEntry{ + OpType: model.OpTypePut, + Key: kvIter.Key(), + Value: kvIter.Value(), + StartTs: h.ts - 1, + CRTs: h.ts + 1, + } + pEvent := model.NewPolymorphicEvent(rawKV) + err := h.mounter.DecodeEvent(context.Background(), pEvent) + require.Nil(h.t, err) + if pEvent.Row == nil { + return ret + } + + row := pEvent.Row + ret = append(ret, row) + + err = kvIter.Next() + require.Nil(h.t, err) + } + + return ret +} + +func requireDebeziumJSONEq(t *testing.T, dbzOutput []byte, tiCDCOutput []byte) { + var ( + ignoredRecordPaths = map[string]bool{ + `{map[string]any}["schema"]`: true, + `{map[string]any}["payload"].(map[string]any)["source"]`: true, + `{map[string]any}["payload"].(map[string]any)["ts_ms"]`: true, + } + + compareOpt = cmp.FilterPath( + func(p cmp.Path) bool { + path := p.GoString() + _, shouldIgnore := ignoredRecordPaths[path] + return shouldIgnore + }, + cmp.Ignore(), + ) + ) + + var objDbzOutput map[string]any + require.Nil(t, json.Unmarshal(dbzOutput, &objDbzOutput), "Failed to unmarshal Debezium JSON") + + var objTiCDCOutput map[string]any + require.Nil(t, json.Unmarshal(tiCDCOutput, &objTiCDCOutput), "Failed to unmarshal TiCDC JSON") + + if diff := cmp.Diff(objDbzOutput, objTiCDCOutput, compareOpt); diff != "" { + require.Fail(t, "JSON is not equal", "Diff (-debezium, +ticdc):\n"+diff) + t.FailNow() + } +} + +func TestDataTypes(t *testing.T) { + dataDDL, err := os.ReadFile("testdata/datatype.ddl.sql") + require.Nil(t, err) + + dataDML, err := os.ReadFile("testdata/datatype.dml.sql") + require.Nil(t, err) + + dataDbzOutput, err := os.ReadFile("testdata/datatype.dbz.json") + require.Nil(t, err) + + helper := NewSQLTestHelper(t, "foo", string(dataDDL)) + + helper.MustExec(`SET sql_mode='';`) + helper.MustExec(`SET time_zone='utc';`) + helper.MustExec(string(dataDML)) + + rows := helper.ScanTable() + cfg := common.NewConfig(config.ProtocolDebezium) + cfg.TimeZone = time.UTC + encoder := NewBatchEncoderBuilder(cfg).Build() + for _, row := range rows { + err := encoder.AppendRowChangedEvent(context.Background(), "", row, nil) + require.Nil(t, err) + } + + messages := encoder.Build() + require.Len(t, messages, 1) + requireDebeziumJSONEq(t, dataDbzOutput, messages[0].Value) +} diff --git a/pkg/sink/codec/debezium/testdata/datatype.dbz.json b/pkg/sink/codec/debezium/testdata/datatype.dbz.json new file mode 100644 index 00000000000..26c1efc126a --- /dev/null +++ b/pkg/sink/codec/debezium/testdata/datatype.dbz.json @@ -0,0 +1,617 @@ +{ + "schema": { + "type": "struct", + "fields": [ + { + "type": "struct", + "fields": [ + { "type": "int32", "optional": false, "field": "pk" }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Date", + "version": 1, + "field": "col_d_2023" + }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Date", + "version": 1, + "field": "col_d_1000" + }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Date", + "version": 1, + "field": "col_d_9999" + }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Date", + "version": 1, + "field": "col_d_0000" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.Timestamp", + "version": 1, + "field": "col_dt" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.Timestamp", + "version": 1, + "field": "col_dt_fsp_0" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.Timestamp", + "version": 1, + "field": "col_dt_fsp_1" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTimestamp", + "version": 1, + "field": "col_dt_fsp_4" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTimestamp", + "version": 1, + "field": "col_dt_fsp_6" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTimestamp", + "version": 1, + "field": "col_dt_0000" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_fsp_0" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_fsp_1" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_fsp_4" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_fsp_6" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_neg" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts_fsp_0" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts_fsp_1" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts_fsp_5" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts_fsp_6" + }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Year", + "version": 1, + "field": "col_y" + }, + { "type": "boolean", "optional": true, "field": "col_bit_1" }, + { + "type": "bytes", + "optional": true, + "name": "io.debezium.data.Bits", + "version": 1, + "parameters": { "length": "5" }, + "field": "col_bit_5" + }, + { + "type": "bytes", + "optional": true, + "name": "io.debezium.data.Bits", + "version": 1, + "parameters": { "length": "6" }, + "field": "col_bit_6" + }, + { + "type": "bytes", + "optional": true, + "name": "io.debezium.data.Bits", + "version": 1, + "parameters": { "length": "60" }, + "field": "col_bit_60" + }, + { "type": "string", "optional": true, "field": "col_varchar" }, + { "type": "string", "optional": true, "field": "col_char" }, + { "type": "string", "optional": true, "field": "col_varbinary" }, + { "type": "string", "optional": true, "field": "col_binary" }, + { "type": "string", "optional": true, "field": "col_blob" }, + { "type": "double", "optional": true, "field": "col_decimal" }, + { "type": "double", "optional": true, "field": "col_numeric" }, + { "type": "float", "optional": true, "field": "col_float" }, + { "type": "double", "optional": true, "field": "col_double" }, + { "type": "int32", "optional": true, "field": "col_int" }, + { "type": "int64", "optional": true, "field": "col_int_unsigned" }, + { "type": "int16", "optional": true, "field": "col_tinyint" }, + { + "type": "int16", + "optional": true, + "field": "col_tinyint_unsigned" + }, + { "type": "int16", "optional": true, "field": "col_smallint" }, + { + "type": "int32", + "optional": true, + "field": "col_smallint_unsigned" + }, + { "type": "int32", "optional": true, "field": "col_mediumint" }, + { + "type": "int32", + "optional": true, + "field": "col_mediumint_unsigned" + }, + { "type": "int64", "optional": true, "field": "col_bigint" }, + { "type": "int64", "optional": true, "field": "col_bigint_unsigned" }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.Enum", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_enum" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.EnumSet", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_set" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.Json", + "version": 1, + "field": "col_json" + } + ], + "optional": true, + "name": "dbserver1.test.foo.Value", + "field": "before" + }, + { + "type": "struct", + "fields": [ + { "type": "int32", "optional": false, "field": "pk" }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Date", + "version": 1, + "field": "col_d_2023" + }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Date", + "version": 1, + "field": "col_d_1000" + }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Date", + "version": 1, + "field": "col_d_9999" + }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Date", + "version": 1, + "field": "col_d_0000" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.Timestamp", + "version": 1, + "field": "col_dt" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.Timestamp", + "version": 1, + "field": "col_dt_fsp_0" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.Timestamp", + "version": 1, + "field": "col_dt_fsp_1" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTimestamp", + "version": 1, + "field": "col_dt_fsp_4" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTimestamp", + "version": 1, + "field": "col_dt_fsp_6" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTimestamp", + "version": 1, + "field": "col_dt_0000" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_fsp_0" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_fsp_1" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_fsp_4" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_fsp_6" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "field": "col_t_neg" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts_fsp_0" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts_fsp_1" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts_fsp_5" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "col_ts_fsp_6" + }, + { + "type": "int32", + "optional": true, + "name": "io.debezium.time.Year", + "version": 1, + "field": "col_y" + }, + { "type": "boolean", "optional": true, "field": "col_bit_1" }, + { + "type": "bytes", + "optional": true, + "name": "io.debezium.data.Bits", + "version": 1, + "parameters": { "length": "5" }, + "field": "col_bit_5" + }, + { + "type": "bytes", + "optional": true, + "name": "io.debezium.data.Bits", + "version": 1, + "parameters": { "length": "6" }, + "field": "col_bit_6" + }, + { + "type": "bytes", + "optional": true, + "name": "io.debezium.data.Bits", + "version": 1, + "parameters": { "length": "60" }, + "field": "col_bit_60" + }, + { "type": "string", "optional": true, "field": "col_varchar" }, + { "type": "string", "optional": true, "field": "col_char" }, + { "type": "string", "optional": true, "field": "col_varbinary" }, + { "type": "string", "optional": true, "field": "col_binary" }, + { "type": "string", "optional": true, "field": "col_blob" }, + { "type": "double", "optional": true, "field": "col_decimal" }, + { "type": "double", "optional": true, "field": "col_numeric" }, + { "type": "float", "optional": true, "field": "col_float" }, + { "type": "double", "optional": true, "field": "col_double" }, + { "type": "int32", "optional": true, "field": "col_int" }, + { "type": "int64", "optional": true, "field": "col_int_unsigned" }, + { "type": "int16", "optional": true, "field": "col_tinyint" }, + { + "type": "int16", + "optional": true, + "field": "col_tinyint_unsigned" + }, + { "type": "int16", "optional": true, "field": "col_smallint" }, + { + "type": "int32", + "optional": true, + "field": "col_smallint_unsigned" + }, + { "type": "int32", "optional": true, "field": "col_mediumint" }, + { + "type": "int32", + "optional": true, + "field": "col_mediumint_unsigned" + }, + { "type": "int64", "optional": true, "field": "col_bigint" }, + { "type": "int64", "optional": true, "field": "col_bigint_unsigned" }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.Enum", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_enum" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.EnumSet", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_set" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.Json", + "version": 1, + "field": "col_json" + } + ], + "optional": true, + "name": "dbserver1.test.foo.Value", + "field": "after" + }, + { + "type": "struct", + "fields": [ + { "type": "string", "optional": false, "field": "version" }, + { "type": "string", "optional": false, "field": "connector" }, + { "type": "string", "optional": false, "field": "name" }, + { "type": "int64", "optional": false, "field": "ts_ms" }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.Enum", + "version": 1, + "parameters": { "allowed": "true,last,false,incremental" }, + "default": "false", + "field": "snapshot" + }, + { "type": "string", "optional": false, "field": "db" }, + { "type": "string", "optional": true, "field": "sequence" }, + { "type": "string", "optional": true, "field": "table" }, + { "type": "int64", "optional": false, "field": "server_id" }, + { "type": "string", "optional": true, "field": "gtid" }, + { "type": "string", "optional": false, "field": "file" }, + { "type": "int64", "optional": false, "field": "pos" }, + { "type": "int32", "optional": false, "field": "row" }, + { "type": "int64", "optional": true, "field": "thread" }, + { "type": "string", "optional": true, "field": "query" } + ], + "optional": false, + "name": "io.debezium.connector.mysql.Source", + "field": "source" + }, + { "type": "string", "optional": false, "field": "op" }, + { "type": "int64", "optional": true, "field": "ts_ms" }, + { + "type": "struct", + "fields": [ + { "type": "string", "optional": false, "field": "id" }, + { "type": "int64", "optional": false, "field": "total_order" }, + { + "type": "int64", + "optional": false, + "field": "data_collection_order" + } + ], + "optional": true, + "name": "event.block", + "version": 1, + "field": "transaction" + } + ], + "optional": false, + "name": "dbserver1.test.foo.Envelope", + "version": 1 + }, + "payload": { + "before": null, + "after": { + "pk": 1, + "col_d_2023": 19691, + "col_d_1000": -354285, + "col_d_9999": 2932896, + "col_d_0000": null, + "col_dt": 1701347696000, + "col_dt_fsp_0": 1701347696000, + "col_dt_fsp_1": 1701347696100, + "col_dt_fsp_4": 1701347696123500, + "col_dt_fsp_6": 1701347696123456, + "col_dt_0000": null, + "col_t": 45296000000, + "col_t_fsp_0": 45296000000, + "col_t_fsp_1": 45296100000, + "col_t_fsp_4": 45296123500, + "col_t_fsp_6": 45296123456, + "col_t_neg": -45296123456, + "col_ts": "2023-11-30T12:34:56Z", + "col_ts_fsp_0": "2023-11-30T12:34:56Z", + "col_ts_fsp_1": "2023-11-30T12:34:56.1Z", + "col_ts_fsp_5": "2023-11-30T12:34:56.12346Z", + "col_ts_fsp_6": "2023-11-30T12:34:56.123456Z", + "col_y": 2023, + "col_bit_1": true, + "col_bit_5": "EA==", + "col_bit_6": "EA==", + "col_bit_60": "EAAAAAAAAAA=", + "col_varchar": "foo", + "col_char": "foo", + "col_varbinary": "Zm9v", + "col_binary": "Zm9vAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==", + "col_blob": "Zm9v", + "col_decimal": 12345.12345, + "col_numeric": 12345.12345, + "col_float": 12345.123, + "col_double": 12345.12345, + "col_int": -2147483648, + "col_int_unsigned": 4294967295, + "col_tinyint": -128, + "col_tinyint_unsigned": 255, + "col_smallint": -32768, + "col_smallint_unsigned": 65535, + "col_mediumint": -8388608, + "col_mediumint_unsigned": 16777215, + "col_bigint": -9223372036854775808, + "col_bigint_unsigned": -1, + "col_enum": "", + "col_set": "a,b", + "col_json": "[\"foo\"]" + }, + "source": { + "version": "2.4.0.Final", + "connector": "mysql", + "name": "dbserver1", + "ts_ms": 1701672687000, + "snapshot": "false", + "db": "test", + "sequence": null, + "table": "foo", + "server_id": 223344, + "gtid": null, + "file": "mysql-bin.000005", + "pos": 7352, + "row": 0, + "thread": 11, + "query": null + }, + "op": "c", + "ts_ms": 1701672687217, + "transaction": null + } +} diff --git a/pkg/sink/codec/debezium/testdata/datatype.ddl.sql b/pkg/sink/codec/debezium/testdata/datatype.ddl.sql new file mode 100644 index 00000000000..b9160d53168 --- /dev/null +++ b/pkg/sink/codec/debezium/testdata/datatype.ddl.sql @@ -0,0 +1,62 @@ +CREATE TABLE foo( + pk INT PRIMARY KEY, + + col_d_2023 DATE, + col_d_1000 DATE, + col_d_9999 DATE, + col_d_0000 DATE, + + col_dt DATETIME, + col_dt_fsp_0 DATETIME(0), + col_dt_fsp_1 DATETIME(1), + col_dt_fsp_4 DATETIME(4), + col_dt_fsp_6 DATETIME(6), + col_dt_0000 DATETIME(6), + + col_t TIME, + col_t_fsp_0 TIME(0), + col_t_fsp_1 TIME(1), + col_t_fsp_4 TIME(4), + col_t_fsp_6 TIME(6), + col_t_neg TIME(6), + + col_ts TIMESTAMP, + col_ts_fsp_0 TIMESTAMP(0), + col_ts_fsp_1 TIMESTAMP(1), + col_ts_fsp_5 TIMESTAMP(5), + col_ts_fsp_6 TIMESTAMP(6), + + col_y YEAR, + + col_bit_1 BIT(1), + col_bit_5 BIT(5), + col_bit_6 BIT(6), + col_bit_60 BIT(60), + + col_varchar VARCHAR(100), + col_char CHAR(100), + col_varbinary VARBINARY(100), + col_binary BINARY(100), + col_blob BLOB, + + col_decimal DECIMAL(10, 5), + col_numeric NUMERIC(10, 5), + col_float FLOAT, + col_double DOUBLE, + + col_int INT, + col_int_unsigned INT UNSIGNED, + col_tinyint TINYINT, + col_tinyint_unsigned TINYINT UNSIGNED, + col_smallint SMALLINT, + col_smallint_unsigned SMALLINT UNSIGNED, + col_mediumint MEDIUMINT, + col_mediumint_unsigned MEDIUMINT UNSIGNED, + col_bigint BIGINT, + col_bigint_unsigned BIGINT UNSIGNED, + + col_enum ENUM('a', 'b', 'c'), + col_set SET('a', 'b', 'c'), + + col_json JSON +); diff --git a/pkg/sink/codec/debezium/testdata/datatype.dml.sql b/pkg/sink/codec/debezium/testdata/datatype.dml.sql new file mode 100644 index 00000000000..c99d3a47309 --- /dev/null +++ b/pkg/sink/codec/debezium/testdata/datatype.dml.sql @@ -0,0 +1,67 @@ +INSERT INTO foo VALUES ( + 1, + + '2023-11-30', + '1000-01-01', + '9999-12-31', + '0000-00-00', + + '2023-11-30 12:34:56.123456', + '2023-11-30 12:34:56.123456', + '2023-11-30 12:34:56.123456', + '2023-11-30 12:34:56.123456', + '2023-11-30 12:34:56.123456', + '0000-00-00 00:00:00.000000', + + '12:34:56.123456', + '12:34:56.123456', + '12:34:56.123456', + '12:34:56.123456', + '12:34:56.123456', + '-12:34:56.123456', + + '2023-11-30 12:34:56.123456', + '2023-11-30 12:34:56.123456', + '2023-11-30 12:34:56.123456', + '2023-11-30 12:34:56.123456', + '2023-11-30 12:34:56.123456', + + '2023', + + 1, + 16, + 16, + 16, + + 'foo', + 'foo', + 'foo', + 'foo', + 'foo', + + 12345.12345, + 12345.12345, + + 12345.12345, + 12345.12345, + + -2147483648, + 4294967295, + + -128, + 255, + + -32768, + 65535, + + -8388608, + 16777215, + + -9223372036854775808, + 18446744073709551615, + + 'a,b', + 'a,b', + + '["foo"]' +); diff --git a/tests/integration_tests/debezium/sql/data_types.sql b/tests/integration_tests/debezium/sql/data_types.sql index 6d3096c2777..d43171fa207 100644 --- a/tests/integration_tests/debezium/sql/data_types.sql +++ b/tests/integration_tests/debezium/sql/data_types.sql @@ -399,3 +399,7 @@ CREATE TABLE t_enum( ); INSERT INTO t_enum VALUES ('a', 'c', 1); + +SET sql_mode=''; +INSERT INTO t_enum VALUES ('d', 'e'); +SET sql_mode='strict_trans_tables'; From e7a37245705f9dfe47275ff0ab93471a1f994a39 Mon Sep 17 00:00:00 2001 From: Wish Date: Mon, 4 Dec 2023 15:10:51 +0800 Subject: [PATCH 08/25] Add more enum cases Signed-off-by: Wish --- .../codec/debezium/testdata/datatype.dbz.json | 42 +++++- .../codec/debezium/testdata/datatype.ddl.sql | 122 +++++++++--------- .../codec/debezium/testdata/datatype.dml.sql | 4 +- 3 files changed, 103 insertions(+), 65 deletions(-) diff --git a/pkg/sink/codec/debezium/testdata/datatype.dbz.json b/pkg/sink/codec/debezium/testdata/datatype.dbz.json index 26c1efc126a..0b47f2362c1 100644 --- a/pkg/sink/codec/debezium/testdata/datatype.dbz.json +++ b/pkg/sink/codec/debezium/testdata/datatype.dbz.json @@ -224,6 +224,14 @@ "parameters": { "allowed": "a,b,c" }, "field": "col_enum" }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.Enum", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_enum_invalid" + }, { "type": "string", "optional": true, @@ -232,6 +240,14 @@ "parameters": { "allowed": "a,b,c" }, "field": "col_set" }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.EnumSet", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_set_invalid" + }, { "type": "string", "optional": true, @@ -466,6 +482,14 @@ "parameters": { "allowed": "a,b,c" }, "field": "col_enum" }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.Enum", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_enum_invalid" + }, { "type": "string", "optional": true, @@ -474,6 +498,14 @@ "parameters": { "allowed": "a,b,c" }, "field": "col_set" }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.EnumSet", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_set_invalid" + }, { "type": "string", "optional": true, @@ -589,15 +621,17 @@ "col_mediumint_unsigned": 16777215, "col_bigint": -9223372036854775808, "col_bigint_unsigned": -1, - "col_enum": "", + "col_enum": "a", + "col_enum_invalid": "", "col_set": "a,b", + "col_set_invalid": "", "col_json": "[\"foo\"]" }, "source": { "version": "2.4.0.Final", "connector": "mysql", "name": "dbserver1", - "ts_ms": 1701672687000, + "ts_ms": 1701673705000, "snapshot": "false", "db": "test", "sequence": null, @@ -605,13 +639,13 @@ "server_id": 223344, "gtid": null, "file": "mysql-bin.000005", - "pos": 7352, + "pos": 10394, "row": 0, "thread": 11, "query": null }, "op": "c", - "ts_ms": 1701672687217, + "ts_ms": 1701673705263, "transaction": null } } diff --git a/pkg/sink/codec/debezium/testdata/datatype.ddl.sql b/pkg/sink/codec/debezium/testdata/datatype.ddl.sql index b9160d53168..0381b43ad08 100644 --- a/pkg/sink/codec/debezium/testdata/datatype.ddl.sql +++ b/pkg/sink/codec/debezium/testdata/datatype.ddl.sql @@ -1,62 +1,64 @@ CREATE TABLE foo( - pk INT PRIMARY KEY, - - col_d_2023 DATE, - col_d_1000 DATE, - col_d_9999 DATE, - col_d_0000 DATE, - - col_dt DATETIME, - col_dt_fsp_0 DATETIME(0), - col_dt_fsp_1 DATETIME(1), - col_dt_fsp_4 DATETIME(4), - col_dt_fsp_6 DATETIME(6), - col_dt_0000 DATETIME(6), - - col_t TIME, - col_t_fsp_0 TIME(0), - col_t_fsp_1 TIME(1), - col_t_fsp_4 TIME(4), - col_t_fsp_6 TIME(6), - col_t_neg TIME(6), - - col_ts TIMESTAMP, - col_ts_fsp_0 TIMESTAMP(0), - col_ts_fsp_1 TIMESTAMP(1), - col_ts_fsp_5 TIMESTAMP(5), - col_ts_fsp_6 TIMESTAMP(6), - - col_y YEAR, - - col_bit_1 BIT(1), - col_bit_5 BIT(5), - col_bit_6 BIT(6), - col_bit_60 BIT(60), - - col_varchar VARCHAR(100), - col_char CHAR(100), - col_varbinary VARBINARY(100), - col_binary BINARY(100), - col_blob BLOB, - - col_decimal DECIMAL(10, 5), - col_numeric NUMERIC(10, 5), - col_float FLOAT, - col_double DOUBLE, - - col_int INT, - col_int_unsigned INT UNSIGNED, - col_tinyint TINYINT, - col_tinyint_unsigned TINYINT UNSIGNED, - col_smallint SMALLINT, - col_smallint_unsigned SMALLINT UNSIGNED, - col_mediumint MEDIUMINT, - col_mediumint_unsigned MEDIUMINT UNSIGNED, - col_bigint BIGINT, - col_bigint_unsigned BIGINT UNSIGNED, - - col_enum ENUM('a', 'b', 'c'), - col_set SET('a', 'b', 'c'), - - col_json JSON + pk INT PRIMARY KEY, + + col_d_2023 DATE, + col_d_1000 DATE, + col_d_9999 DATE, + col_d_0000 DATE, + + col_dt DATETIME, + col_dt_fsp_0 DATETIME(0), + col_dt_fsp_1 DATETIME(1), + col_dt_fsp_4 DATETIME(4), + col_dt_fsp_6 DATETIME(6), + col_dt_0000 DATETIME(6), + + col_t TIME, + col_t_fsp_0 TIME(0), + col_t_fsp_1 TIME(1), + col_t_fsp_4 TIME(4), + col_t_fsp_6 TIME(6), + col_t_neg TIME(6), + + col_ts TIMESTAMP, + col_ts_fsp_0 TIMESTAMP(0), + col_ts_fsp_1 TIMESTAMP(1), + col_ts_fsp_5 TIMESTAMP(5), + col_ts_fsp_6 TIMESTAMP(6), + + col_y YEAR, + + col_bit_1 BIT(1), + col_bit_5 BIT(5), + col_bit_6 BIT(6), + col_bit_60 BIT(60), + + col_varchar VARCHAR(100), + col_char CHAR(100), + col_varbinary VARBINARY(100), + col_binary BINARY(100), + col_blob BLOB, + + col_decimal DECIMAL(10, 5), + col_numeric NUMERIC(10, 5), + col_float FLOAT, + col_double DOUBLE, + + col_int INT, + col_int_unsigned INT UNSIGNED, + col_tinyint TINYINT, + col_tinyint_unsigned TINYINT UNSIGNED, + col_smallint SMALLINT, + col_smallint_unsigned SMALLINT UNSIGNED, + col_mediumint MEDIUMINT, + col_mediumint_unsigned MEDIUMINT UNSIGNED, + col_bigint BIGINT, + col_bigint_unsigned BIGINT UNSIGNED, + + col_enum ENUM('a', 'b', 'c'), + col_enum_invalid ENUM('a', 'b', 'c'), + col_set SET('a', 'b', 'c'), + col_set_invalid SET('a', 'b', 'c'), + + col_json JSON ); diff --git a/pkg/sink/codec/debezium/testdata/datatype.dml.sql b/pkg/sink/codec/debezium/testdata/datatype.dml.sql index c99d3a47309..b716f80d080 100644 --- a/pkg/sink/codec/debezium/testdata/datatype.dml.sql +++ b/pkg/sink/codec/debezium/testdata/datatype.dml.sql @@ -60,8 +60,10 @@ INSERT INTO foo VALUES ( -9223372036854775808, 18446744073709551615, + 'a', + 'd', 'a,b', - 'a,b', + 'd', '["foo"]' ); From 72d8718e2f4e5d9d44b0302ccf6c87e00e27a169 Mon Sep 17 00:00:00 2001 From: Wish Date: Mon, 4 Dec 2023 15:19:44 +0800 Subject: [PATCH 09/25] Address comments Signed-off-by: Wish --- pkg/sink/codec/debezium/codec.go | 111 ++++++++++++------------------- 1 file changed, 44 insertions(+), 67 deletions(-) diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index de77f3605bb..104201f1ab0 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/errors" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/util" @@ -81,12 +80,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f return nil } } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for bit column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for bit column %s", + col.Value, + col.Name) case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: if col.Flag.IsBinary() { @@ -94,23 +91,19 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f c.writeBinaryField(writer, col.Name, v) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for binary string column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for binary string column %s", + col.Value, + col.Name) } else { if v, ok := col.Value.([]byte); ok { writer.WriteStringField(col.Name, string(hack.String(v))) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for non-binary string column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for non-binary string column %s", + col.Value, + col.Name) } case mysql.TypeEnum: if v, ok := col.Value.(uint64); ok { @@ -123,12 +116,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f writer.WriteStringField(col.Name, enumVar.Name) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for enum column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for enum column %s", + col.Value, + col.Name) case mysql.TypeSet: if v, ok := col.Value.(uint64); ok { setVar, err := types.ParseSetValue(ft.GetElems(), v) @@ -140,12 +131,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f writer.WriteStringField(col.Name, setVar.Name) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for set column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for set column %s", + col.Value, + col.Name) case mysql.TypeNewDecimal: if v, ok := col.Value.(string); ok { floatV, err := strconv.ParseFloat(v, 64) @@ -157,12 +146,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f writer.WriteFloat64Field(col.Name, floatV) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for decimal column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for decimal column %s", + col.Value, + col.Name) case mysql.TypeDate, mysql.TypeNewDate: if v, ok := col.Value.(string); ok { t, err := time.Parse("2006-01-02", v) @@ -180,12 +167,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f writer.WriteInt64Field(col.Name, t.Unix()/60/60/24) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for date column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for date column %s", + col.Value, + col.Name) case mysql.TypeDatetime: // Debezium behavior from doc: // > Such columns are converted into epoch milliseconds or microseconds based on the @@ -212,12 +197,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f return nil } } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for datetime column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for datetime column %s", + col.Value, + col.Name) case mysql.TypeTimestamp: // Debezium behavior from doc: // > The TIMESTAMP type represents a timestamp without time zone information. @@ -251,12 +234,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f writer.WriteStringField(col.Name, str) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for timestamp column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for timestamp column %s", + col.Value, + col.Name) case mysql.TypeDuration: // Debezium behavior from doc: // > Represents the time value in microseconds and does not include @@ -273,12 +254,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f writer.WriteInt64Field(col.Name, d.Microseconds()) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for time column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for time column %s", + col.Value, + col.Name) case mysql.TypeLonglong: if col.Flag.IsUnsigned() { // Handle with BIGINT UNSIGNED. @@ -287,12 +266,10 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f writer.WriteInt64Field(col.Name, int64(v)) return nil } - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - errors.Errorf( - "unexpected column value type %T for unsigned bigint column %s", - col.Value, - col.Name)) + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for unsigned bigint column %s", + col.Value, + col.Name) } // Note: Although Debezium's doc claims to use INT32 for INT, but it From 40e0e32194387326f192965443b913ceec44c2cf Mon Sep 17 00:00:00 2001 From: Wish Date: Mon, 4 Dec 2023 15:47:45 +0800 Subject: [PATCH 10/25] Fix performance issue Signed-off-by: Wish --- pkg/util/json_writer.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go index a1882d53fbd..17c7606fe62 100644 --- a/pkg/util/json_writer.go +++ b/pkg/util/json_writer.go @@ -49,6 +49,7 @@ func BorrowJSONWriter(out io.Writer) *JSONWriter { func ReturnJSONWriter(w *JSONWriter) { w.stream.Flush() + jsonAPI.ReturnStream(w.stream) w.out = nil w.stream = nil jWriterPool.Put(w) From 5f21701baa9a3584954b07b2c47aabdfd734b663 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 5 Dec 2023 10:37:38 +0800 Subject: [PATCH 11/25] Add JSON writer tests Signed-off-by: Wish --- pkg/util/json_writer_test.go | 244 +++++++++++++++++++++++++++++++++++ 1 file changed, 244 insertions(+) create mode 100644 pkg/util/json_writer_test.go diff --git a/pkg/util/json_writer_test.go b/pkg/util/json_writer_test.go new file mode 100644 index 00000000000..7c28ad2c5cc --- /dev/null +++ b/pkg/util/json_writer_test.go @@ -0,0 +1,244 @@ +// Copyright 2024 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 util + +import ( + "bytes" + "testing" + + "github.com/stretchr/testify/require" +) + +func writeJSON(fn func(*JSONWriter)) string { + out := &bytes.Buffer{} + w := BorrowJSONWriter(out) + fn(w) + ReturnJSONWriter(w) + return out.String() +} + +func TestObject(t *testing.T) { + var s string + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() {}) + }) + require.Equal(t, `{}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteAnyField("foo", 1) + }) + }) + require.Equal(t, `{"foo":1}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteAnyField("foo", 1) + w.WriteAnyField("bar", 2) + }) + }) + require.Equal(t, `{"foo":1,"bar":2}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() {}) + }) + }) + require.Equal(t, `{"foo":{}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() {}) + w.WriteObjectField("bar", func() {}) + }) + }) + require.Equal(t, `{"foo":{},"bar":{}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteObjectField("foo1", func() {}) + }) + w.WriteObjectField("bar", func() { + w.WriteObjectField("foo2", func() {}) + }) + }) + }) + require.Equal(t, `{"foo":{"foo1":{}},"bar":{"foo2":{}}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteObjectField("foo1", func() {}) + w.WriteObjectField("bar1", func() {}) + }) + w.WriteObjectField("bar", func() { + w.WriteObjectField("foo2", func() {}) + }) + }) + }) + require.Equal(t, `{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{}}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteObjectField("foo1", func() {}) + w.WriteObjectField("bar1", func() {}) + }) + w.WriteObjectField("bar", func() { + w.WriteObjectField("foo2", func() {}) + w.WriteObjectField("bar2", func() {}) + }) + }) + }) + require.Equal(t, `{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteObjectField("foo1", func() { + w.WriteNullField("abc") + }) + w.WriteObjectField("bar1", func() {}) + }) + w.WriteObjectField("bar", func() { + w.WriteObjectField("foo2", func() {}) + w.WriteObjectField("bar2", func() {}) + }) + }) + }) + require.Equal(t, `{"foo":{"foo1":{"abc":null},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) +} + +func TestBase64(t *testing.T) { + var s string + + s = writeJSON(func(w *JSONWriter) { + w.WriteBase64String([]byte("foo")) + }) + require.Equal(t, `"Zm9v"`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteBase64StringField("foo", []byte("bar")) + }) + }) + require.Equal(t, `{"foo":"YmFy"}`, s) +} + +func TestField(t *testing.T) { + var s string + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteStringField("foo", "bar") + }) + }) + require.Equal(t, `{"foo":"bar"}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteUint64Field("foo", 1) + }) + }) + require.Equal(t, `{"foo":1}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteFloat64Field("foo", 1.1) + }) + }) + require.Equal(t, `{"foo":1.1}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteNullField("foo") + }) + }) + require.Equal(t, `{"foo":null}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteAnyField("foo", nil) + }) + }) + require.Equal(t, `{"foo":null}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteAnyField("foo", 1) + }) + }) + require.Equal(t, `{"foo":1}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteBoolField("foo", true) + }) + }) + require.Equal(t, `{"foo":true}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteBoolField("foo", false) + }) + }) + require.Equal(t, `{"foo":false}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() {}) + }) + }) + require.Equal(t, `{"foo":{}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteUint64Field("bar", 1) + }) + }) + }) + require.Equal(t, `{"foo":{"bar":1}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteUint64Field("bar", 1) + w.WriteStringField("abc", "def") + }) + }) + }) + require.Equal(t, `{"foo":{"bar":1,"abc":"def"}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteUint64Field("bar", 1) + w.WriteStringField("abc", "def") + w.WriteNullField("xyz") + }) + }) + require.Equal(t, `{"bar":1,"abc":"def","xyz":null}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteUint64Field("bar", 1) + w.WriteStringField("abc", "def") + w.WriteNullField("xyz") + w.WriteObjectField("foo", func() {}) + }) + }) + require.Equal(t, `{"bar":1,"abc":"def","xyz":null,"foo":{}}`, s) +} From bc9c6a4737ea1dbd424e0f14ad3cf6102b9b1c34 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 5 Dec 2023 10:40:52 +0800 Subject: [PATCH 12/25] util: Add JSONWriter Signed-off-by: Wish --- pkg/util/json_writer.go | 175 +++++++++++++++++++++++++ pkg/util/json_writer_test.go | 244 +++++++++++++++++++++++++++++++++++ 2 files changed, 419 insertions(+) create mode 100644 pkg/util/json_writer.go create mode 100644 pkg/util/json_writer_test.go diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go new file mode 100644 index 00000000000..17c7606fe62 --- /dev/null +++ b/pkg/util/json_writer.go @@ -0,0 +1,175 @@ +// Copyright 2024 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 util + +import ( + "encoding/base64" + "io" + "sync" + + jsoniter "github.com/json-iterator/go" +) + +var jsonAPI = jsoniter.Config{ + EscapeHTML: false, + SortMapKeys: false, +}.Froze() + +var jWriterPool = sync.Pool{ + New: func() interface{} { + return &JSONWriter{} + }, +} + +type JSONWriter struct { + out io.Writer + stream *jsoniter.Stream // `stream` is created over `out` + + needPrependComma bool +} + +func BorrowJSONWriter(out io.Writer) *JSONWriter { + w := jWriterPool.Get().(*JSONWriter) + w.out = out + w.stream = jsonAPI.BorrowStream(out) + w.needPrependComma = false + return w +} + +func ReturnJSONWriter(w *JSONWriter) { + w.stream.Flush() + jsonAPI.ReturnStream(w.stream) + w.out = nil + w.stream = nil + jWriterPool.Put(w) +} + +func (w *JSONWriter) WriteBase64String(b []byte) { + // As we write to out directly so we need to flush the jsoniter stream first. + w.stream.Flush() + w.out.Write([]byte(`"`)) + encoder := base64.NewEncoder(base64.StdEncoding, w.out) + _, _ = encoder.Write(b) + encoder.Close() + w.out.Write([]byte(`"`)) +} + +func (w *JSONWriter) WriteObject(objectFieldsWriteFn func()) { + lastNeedPrependComma := w.needPrependComma + w.needPrependComma = false + w.stream.WriteObjectStart() + objectFieldsWriteFn() + w.stream.WriteObjectEnd() + w.needPrependComma = lastNeedPrependComma +} + +func (w *JSONWriter) WriteBoolField(fieldName string, value bool) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.stream.WriteBool(value) +} + +func (w *JSONWriter) WriteIntField(fieldName string, value int) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.stream.WriteInt(value) +} + +func (w *JSONWriter) WriteInt64Field(fieldName string, value int64) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.stream.WriteInt64(value) +} + +func (w *JSONWriter) WriteUint64Field(fieldName string, value uint64) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.stream.WriteUint64(value) +} + +func (w *JSONWriter) WriteFloat64Field(fieldName string, value float64) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.stream.WriteFloat64(value) +} + +func (w *JSONWriter) WriteStringField(fieldName string, value string) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.stream.WriteString(value) +} + +func (w *JSONWriter) WriteBase64StringField(fieldName string, b []byte) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.WriteBase64String(b) +} + +func (w *JSONWriter) WriteAnyField(fieldName string, value any) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.stream.WriteVal(value) +} + +func (w *JSONWriter) WriteObjectField(fieldName string, objectFieldsWriteFn func()) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.WriteObject(objectFieldsWriteFn) +} + +func (w *JSONWriter) WriteNullField(fieldName string) { + if w.needPrependComma { + w.stream.WriteMore() + } else { + w.needPrependComma = true + } + w.stream.WriteObjectField(fieldName) + w.stream.WriteNil() +} diff --git a/pkg/util/json_writer_test.go b/pkg/util/json_writer_test.go new file mode 100644 index 00000000000..7c28ad2c5cc --- /dev/null +++ b/pkg/util/json_writer_test.go @@ -0,0 +1,244 @@ +// Copyright 2024 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 util + +import ( + "bytes" + "testing" + + "github.com/stretchr/testify/require" +) + +func writeJSON(fn func(*JSONWriter)) string { + out := &bytes.Buffer{} + w := BorrowJSONWriter(out) + fn(w) + ReturnJSONWriter(w) + return out.String() +} + +func TestObject(t *testing.T) { + var s string + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() {}) + }) + require.Equal(t, `{}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteAnyField("foo", 1) + }) + }) + require.Equal(t, `{"foo":1}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteAnyField("foo", 1) + w.WriteAnyField("bar", 2) + }) + }) + require.Equal(t, `{"foo":1,"bar":2}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() {}) + }) + }) + require.Equal(t, `{"foo":{}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() {}) + w.WriteObjectField("bar", func() {}) + }) + }) + require.Equal(t, `{"foo":{},"bar":{}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteObjectField("foo1", func() {}) + }) + w.WriteObjectField("bar", func() { + w.WriteObjectField("foo2", func() {}) + }) + }) + }) + require.Equal(t, `{"foo":{"foo1":{}},"bar":{"foo2":{}}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteObjectField("foo1", func() {}) + w.WriteObjectField("bar1", func() {}) + }) + w.WriteObjectField("bar", func() { + w.WriteObjectField("foo2", func() {}) + }) + }) + }) + require.Equal(t, `{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{}}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteObjectField("foo1", func() {}) + w.WriteObjectField("bar1", func() {}) + }) + w.WriteObjectField("bar", func() { + w.WriteObjectField("foo2", func() {}) + w.WriteObjectField("bar2", func() {}) + }) + }) + }) + require.Equal(t, `{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteObjectField("foo1", func() { + w.WriteNullField("abc") + }) + w.WriteObjectField("bar1", func() {}) + }) + w.WriteObjectField("bar", func() { + w.WriteObjectField("foo2", func() {}) + w.WriteObjectField("bar2", func() {}) + }) + }) + }) + require.Equal(t, `{"foo":{"foo1":{"abc":null},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) +} + +func TestBase64(t *testing.T) { + var s string + + s = writeJSON(func(w *JSONWriter) { + w.WriteBase64String([]byte("foo")) + }) + require.Equal(t, `"Zm9v"`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteBase64StringField("foo", []byte("bar")) + }) + }) + require.Equal(t, `{"foo":"YmFy"}`, s) +} + +func TestField(t *testing.T) { + var s string + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteStringField("foo", "bar") + }) + }) + require.Equal(t, `{"foo":"bar"}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteUint64Field("foo", 1) + }) + }) + require.Equal(t, `{"foo":1}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteFloat64Field("foo", 1.1) + }) + }) + require.Equal(t, `{"foo":1.1}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteNullField("foo") + }) + }) + require.Equal(t, `{"foo":null}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteAnyField("foo", nil) + }) + }) + require.Equal(t, `{"foo":null}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteAnyField("foo", 1) + }) + }) + require.Equal(t, `{"foo":1}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteBoolField("foo", true) + }) + }) + require.Equal(t, `{"foo":true}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteBoolField("foo", false) + }) + }) + require.Equal(t, `{"foo":false}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() {}) + }) + }) + require.Equal(t, `{"foo":{}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteUint64Field("bar", 1) + }) + }) + }) + require.Equal(t, `{"foo":{"bar":1}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteObjectField("foo", func() { + w.WriteUint64Field("bar", 1) + w.WriteStringField("abc", "def") + }) + }) + }) + require.Equal(t, `{"foo":{"bar":1,"abc":"def"}}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteUint64Field("bar", 1) + w.WriteStringField("abc", "def") + w.WriteNullField("xyz") + }) + }) + require.Equal(t, `{"bar":1,"abc":"def","xyz":null}`, s) + + s = writeJSON(func(w *JSONWriter) { + w.WriteObject(func() { + w.WriteUint64Field("bar", 1) + w.WriteStringField("abc", "def") + w.WriteNullField("xyz") + w.WriteObjectField("foo", func() {}) + }) + }) + require.Equal(t, `{"bar":1,"abc":"def","xyz":null,"foo":{}}`, s) +} From 5440a237ec80a7aca86abda1c0b251b691b00392 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 5 Dec 2023 21:24:34 +0800 Subject: [PATCH 13/25] Support internal buffers Signed-off-by: Wish --- pkg/util/json_writer.go | 30 ++++-- pkg/util/json_writer_test.go | 190 ++++++++++++++++++++++++----------- 2 files changed, 155 insertions(+), 65 deletions(-) diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go index 17c7606fe62..d9e3a551cbe 100644 --- a/pkg/util/json_writer.go +++ b/pkg/util/json_writer.go @@ -55,14 +55,30 @@ func ReturnJSONWriter(w *JSONWriter) { jWriterPool.Put(w) } +// Buffer returns the buffer if out is nil. +// WARN: You may need to copy the result of the buffer. Otherwise the content of the buffer +// may be changed. +func (w *JSONWriter) Buffer() []byte { + return w.stream.Buffer() +} + func (w *JSONWriter) WriteBase64String(b []byte) { - // As we write to out directly so we need to flush the jsoniter stream first. - w.stream.Flush() - w.out.Write([]byte(`"`)) - encoder := base64.NewEncoder(base64.StdEncoding, w.out) - _, _ = encoder.Write(b) - encoder.Close() - w.out.Write([]byte(`"`)) + if w.out == nil { + w.stream.WriteRaw(`"`) + encoder := base64.NewEncoder(base64.StdEncoding, w.stream) + _, _ = encoder.Write(b) + encoder.Close() + w.stream.WriteRaw(`"`) + } else { + // If out is available, let's write to out directly to avoid extra copy. + // As we write to out directly so we need to flush the jsoniter stream first. + w.stream.Flush() + w.out.Write([]byte(`"`)) + encoder := base64.NewEncoder(base64.StdEncoding, w.out) + _, _ = encoder.Write(b) + encoder.Close() + w.out.Write([]byte(`"`)) + } } func (w *JSONWriter) WriteObject(objectFieldsWriteFn func()) { diff --git a/pkg/util/json_writer_test.go b/pkg/util/json_writer_test.go index 7c28ad2c5cc..11f277a0bfe 100644 --- a/pkg/util/json_writer_test.go +++ b/pkg/util/json_writer_test.go @@ -17,56 +17,70 @@ import ( "bytes" "testing" - "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "github.com/thanhpk/randstr" ) -func writeJSON(fn func(*JSONWriter)) string { - out := &bytes.Buffer{} - w := BorrowJSONWriter(out) - fn(w) - ReturnJSONWriter(w) - return out.String() +type JSONWriterTestSuite struct { + suite.Suite + useInternalBuffer bool +} + +func (s *JSONWriterTestSuite) writeJSON(fn func(*JSONWriter)) string { + if s.useInternalBuffer { + w := BorrowJSONWriter(nil) + fn(w) + ret := string(w.Buffer()) + ReturnJSONWriter(w) + return ret + } else { + out := &bytes.Buffer{} + w := BorrowJSONWriter(out) + fn(w) + ReturnJSONWriter(w) + return out.String() + } } -func TestObject(t *testing.T) { +func (suite *JSONWriterTestSuite) TestObject() { var s string - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() {}) }) - require.Equal(t, `{}`, s) + suite.Require().Equal(`{}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteAnyField("foo", 1) }) }) - require.Equal(t, `{"foo":1}`, s) + suite.Require().Equal(`{"foo":1}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteAnyField("foo", 1) w.WriteAnyField("bar", 2) }) }) - require.Equal(t, `{"foo":1,"bar":2}`, s) + suite.Require().Equal(`{"foo":1,"bar":2}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() {}) }) }) - require.Equal(t, `{"foo":{}}`, s) + suite.Require().Equal(`{"foo":{}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() {}) w.WriteObjectField("bar", func() {}) }) }) - require.Equal(t, `{"foo":{},"bar":{}}`, s) + suite.Require().Equal(`{"foo":{},"bar":{}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteObjectField("foo1", func() {}) @@ -76,9 +90,9 @@ func TestObject(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"foo1":{}},"bar":{"foo2":{}}}`, s) + suite.Require().Equal(`{"foo":{"foo1":{}},"bar":{"foo2":{}}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteObjectField("foo1", func() {}) @@ -89,9 +103,9 @@ func TestObject(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{}}}`, s) + suite.Require().Equal(`{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{}}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteObjectField("foo1", func() {}) @@ -103,9 +117,9 @@ func TestObject(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) + suite.Require().Equal(`{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteObjectField("foo1", func() { @@ -119,101 +133,101 @@ func TestObject(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"foo1":{"abc":null},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) + suite.Require().Equal(`{"foo":{"foo1":{"abc":null},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) } -func TestBase64(t *testing.T) { +func (suite *JSONWriterTestSuite) TestBase64() { var s string - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteBase64String([]byte("foo")) }) - require.Equal(t, `"Zm9v"`, s) + suite.Require().Equal(`"Zm9v"`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteBase64StringField("foo", []byte("bar")) }) }) - require.Equal(t, `{"foo":"YmFy"}`, s) + suite.Require().Equal(`{"foo":"YmFy"}`, s) } -func TestField(t *testing.T) { +func (suite *JSONWriterTestSuite) TestField() { var s string - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteStringField("foo", "bar") }) }) - require.Equal(t, `{"foo":"bar"}`, s) + suite.Require().Equal(`{"foo":"bar"}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteUint64Field("foo", 1) }) }) - require.Equal(t, `{"foo":1}`, s) + suite.Require().Equal(`{"foo":1}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteFloat64Field("foo", 1.1) }) }) - require.Equal(t, `{"foo":1.1}`, s) + suite.Require().Equal(`{"foo":1.1}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteNullField("foo") }) }) - require.Equal(t, `{"foo":null}`, s) + suite.Require().Equal(`{"foo":null}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteAnyField("foo", nil) }) }) - require.Equal(t, `{"foo":null}`, s) + suite.Require().Equal(`{"foo":null}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteAnyField("foo", 1) }) }) - require.Equal(t, `{"foo":1}`, s) + suite.Require().Equal(`{"foo":1}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteBoolField("foo", true) }) }) - require.Equal(t, `{"foo":true}`, s) + suite.Require().Equal(`{"foo":true}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteBoolField("foo", false) }) }) - require.Equal(t, `{"foo":false}`, s) + suite.Require().Equal(`{"foo":false}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() {}) }) }) - require.Equal(t, `{"foo":{}}`, s) + suite.Require().Equal(`{"foo":{}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteUint64Field("bar", 1) }) }) }) - require.Equal(t, `{"foo":{"bar":1}}`, s) + suite.Require().Equal(`{"foo":{"bar":1}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteUint64Field("bar", 1) @@ -221,18 +235,18 @@ func TestField(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"bar":1,"abc":"def"}}`, s) + suite.Require().Equal(`{"foo":{"bar":1,"abc":"def"}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteUint64Field("bar", 1) w.WriteStringField("abc", "def") w.WriteNullField("xyz") }) }) - require.Equal(t, `{"bar":1,"abc":"def","xyz":null}`, s) + suite.Require().Equal(`{"bar":1,"abc":"def","xyz":null}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteUint64Field("bar", 1) w.WriteStringField("abc", "def") @@ -240,5 +254,65 @@ func TestField(t *testing.T) { w.WriteObjectField("foo", func() {}) }) }) - require.Equal(t, `{"bar":1,"abc":"def","xyz":null,"foo":{}}`, s) + suite.Require().Equal(`{"bar":1,"abc":"def","xyz":null,"foo":{}}`, s) +} + +func TestExternalBuffer(t *testing.T) { + suite.Run(t, &JSONWriterTestSuite{useInternalBuffer: false}) +} + +func TestInternalBuffer(t *testing.T) { + suite.Run(t, &JSONWriterTestSuite{useInternalBuffer: true}) +} + +func BenchmarkExternalBufferWriteBase64(b *testing.B) { + out := &bytes.Buffer{} + str := randstr.Bytes(1024) + for i := 0; i < b.N; i++ { + out.Reset() + w := BorrowJSONWriter(out) + w.WriteObject(func() { + w.WriteBase64StringField("foo", str) + }) + _ = out.Bytes() + ReturnJSONWriter(w) + } +} + +func BenchmarkInternalBufferWriteBase64(b *testing.B) { + str := randstr.Bytes(1024) + for i := 0; i < b.N; i++ { + w := BorrowJSONWriter(nil) + w.WriteObject(func() { + w.WriteBase64StringField("foo", str) + }) + _ = w.Buffer() + ReturnJSONWriter(w) + } +} + +func BenchmarkExternalBufferWriteString(b *testing.B) { + out := &bytes.Buffer{} + str := randstr.String(1024) + for i := 0; i < b.N; i++ { + out.Reset() + w := BorrowJSONWriter(out) + w.WriteObject(func() { + w.WriteStringField("foo", str) + }) + _ = out.Bytes() + ReturnJSONWriter(w) + } +} + +func BenchmarkInternalBufferWriteString(b *testing.B) { + str := randstr.String(1024) + for i := 0; i < b.N; i++ { + w := BorrowJSONWriter(nil) + w.WriteObject(func() { + w.WriteStringField("foo", str) + }) + _ = w.Buffer() + ReturnJSONWriter(w) + } } From 80180a8797ffd1710442dbec2377c50dbb39e58f Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 5 Dec 2023 21:24:44 +0800 Subject: [PATCH 14/25] Fix batch message Signed-off-by: Wish --- pkg/sink/codec/debezium/encoder.go | 69 +++++++++++++----------------- 1 file changed, 30 insertions(+), 39 deletions(-) diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go index 0fd94e28710..944d6b2a519 100644 --- a/pkg/sink/codec/debezium/encoder.go +++ b/pkg/sink/codec/debezium/encoder.go @@ -16,7 +16,6 @@ package debezium import ( "bytes" "context" - "encoding/binary" "time" "github.com/pingcap/tiflow/cdc/model" @@ -27,10 +26,7 @@ import ( ) type BatchEncoder struct { - keyBuf *bytes.Buffer - valueBuf *bytes.Buffer - callbackBuf []func() - batchSize int + messages []*common.Message config *common.Config codec *Codec @@ -49,14 +45,33 @@ func (d *BatchEncoder) AppendRowChangedEvent( e *model.RowChangedEvent, callback func(), ) error { - err := d.codec.EncodeRowChangedEvent(e, d.valueBuf) + valueBuf := bytes.Buffer{} + err := d.codec.EncodeRowChangedEvent(e, &valueBuf) if err != nil { return errors.Trace(err) } - d.batchSize++ - if callback != nil { - d.callbackBuf = append(d.callbackBuf, callback) + // TODO: Use a streaming compression is better. + value, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + valueBuf.Bytes(), + ) + if err != nil { + return errors.Trace(err) } + m := &common.Message{ + Key: nil, + Value: value, + Ts: e.CommitTs, + Schema: &e.Table.Schema, + Table: &e.Table.Table, + Type: model.MessageTypeRow, + Protocol: config.ProtocolDebezium, + Callback: callback, + } + m.IncRowsCount() + + d.messages = append(d.messages, m) return nil } @@ -69,50 +84,26 @@ func (d *BatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message, error // Build implements the RowEventEncoder interface func (d *BatchEncoder) Build() []*common.Message { - if d.batchSize == 0 { + if len(d.messages) == 0 { return nil } - ret := common.NewMsg(config.ProtocolDebezium, - d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MessageTypeRow, nil, nil) - ret.SetRowsCount(d.batchSize) - if len(d.callbackBuf) != 0 && len(d.callbackBuf) == d.batchSize { - callbacks := d.callbackBuf - ret.Callback = func() { - for _, cb := range callbacks { - cb() - } - } - d.callbackBuf = make([]func(), 0) - } - d.reset() - return []*common.Message{ret} -} - -// reset implements the RowEventEncoder interface -func (d *BatchEncoder) reset() { - d.keyBuf.Reset() - d.valueBuf.Reset() - d.batchSize = 0 - var versionByte [8]byte - binary.BigEndian.PutUint64(versionByte[:], codec.BatchVersion1) - d.keyBuf.Write(versionByte[:]) + result := d.messages + d.messages = nil + return result } // newBatchEncoder creates a new Debezium BatchEncoder. func newBatchEncoder(c *common.Config) codec.RowEventEncoder { batch := &BatchEncoder{ - keyBuf: &bytes.Buffer{}, - valueBuf: &bytes.Buffer{}, - callbackBuf: make([]func(), 0), - config: c, + messages: nil, + config: c, codec: &Codec{ config: c, clusterID: config.GetGlobalServerConfig().ClusterID, nowFunc: time.Now, }, } - batch.reset() return batch } From 5c3c13d0b7cbb658386c0fd28cf07796225b78ec Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 5 Dec 2023 21:30:11 +0800 Subject: [PATCH 15/25] Fix integration test Signed-off-by: Wish --- tests/integration_tests/debezium/sql/data_types.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/debezium/sql/data_types.sql b/tests/integration_tests/debezium/sql/data_types.sql index d43171fa207..48452db04fe 100644 --- a/tests/integration_tests/debezium/sql/data_types.sql +++ b/tests/integration_tests/debezium/sql/data_types.sql @@ -401,5 +401,5 @@ CREATE TABLE t_enum( INSERT INTO t_enum VALUES ('a', 'c', 1); SET sql_mode=''; -INSERT INTO t_enum VALUES ('d', 'e'); +INSERT INTO t_enum VALUES ('d', 'e', 2); SET sql_mode='strict_trans_tables'; From c19795ed144fb94c67839d3e5ede56fd0a77df09 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 5 Dec 2023 21:24:34 +0800 Subject: [PATCH 16/25] Support internal buffers Signed-off-by: Wish --- pkg/util/json_writer.go | 30 ++++-- pkg/util/json_writer_test.go | 190 ++++++++++++++++++++++++----------- 2 files changed, 155 insertions(+), 65 deletions(-) diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go index 17c7606fe62..d9e3a551cbe 100644 --- a/pkg/util/json_writer.go +++ b/pkg/util/json_writer.go @@ -55,14 +55,30 @@ func ReturnJSONWriter(w *JSONWriter) { jWriterPool.Put(w) } +// Buffer returns the buffer if out is nil. +// WARN: You may need to copy the result of the buffer. Otherwise the content of the buffer +// may be changed. +func (w *JSONWriter) Buffer() []byte { + return w.stream.Buffer() +} + func (w *JSONWriter) WriteBase64String(b []byte) { - // As we write to out directly so we need to flush the jsoniter stream first. - w.stream.Flush() - w.out.Write([]byte(`"`)) - encoder := base64.NewEncoder(base64.StdEncoding, w.out) - _, _ = encoder.Write(b) - encoder.Close() - w.out.Write([]byte(`"`)) + if w.out == nil { + w.stream.WriteRaw(`"`) + encoder := base64.NewEncoder(base64.StdEncoding, w.stream) + _, _ = encoder.Write(b) + encoder.Close() + w.stream.WriteRaw(`"`) + } else { + // If out is available, let's write to out directly to avoid extra copy. + // As we write to out directly so we need to flush the jsoniter stream first. + w.stream.Flush() + w.out.Write([]byte(`"`)) + encoder := base64.NewEncoder(base64.StdEncoding, w.out) + _, _ = encoder.Write(b) + encoder.Close() + w.out.Write([]byte(`"`)) + } } func (w *JSONWriter) WriteObject(objectFieldsWriteFn func()) { diff --git a/pkg/util/json_writer_test.go b/pkg/util/json_writer_test.go index 7c28ad2c5cc..11f277a0bfe 100644 --- a/pkg/util/json_writer_test.go +++ b/pkg/util/json_writer_test.go @@ -17,56 +17,70 @@ import ( "bytes" "testing" - "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "github.com/thanhpk/randstr" ) -func writeJSON(fn func(*JSONWriter)) string { - out := &bytes.Buffer{} - w := BorrowJSONWriter(out) - fn(w) - ReturnJSONWriter(w) - return out.String() +type JSONWriterTestSuite struct { + suite.Suite + useInternalBuffer bool +} + +func (s *JSONWriterTestSuite) writeJSON(fn func(*JSONWriter)) string { + if s.useInternalBuffer { + w := BorrowJSONWriter(nil) + fn(w) + ret := string(w.Buffer()) + ReturnJSONWriter(w) + return ret + } else { + out := &bytes.Buffer{} + w := BorrowJSONWriter(out) + fn(w) + ReturnJSONWriter(w) + return out.String() + } } -func TestObject(t *testing.T) { +func (suite *JSONWriterTestSuite) TestObject() { var s string - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() {}) }) - require.Equal(t, `{}`, s) + suite.Require().Equal(`{}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteAnyField("foo", 1) }) }) - require.Equal(t, `{"foo":1}`, s) + suite.Require().Equal(`{"foo":1}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteAnyField("foo", 1) w.WriteAnyField("bar", 2) }) }) - require.Equal(t, `{"foo":1,"bar":2}`, s) + suite.Require().Equal(`{"foo":1,"bar":2}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() {}) }) }) - require.Equal(t, `{"foo":{}}`, s) + suite.Require().Equal(`{"foo":{}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() {}) w.WriteObjectField("bar", func() {}) }) }) - require.Equal(t, `{"foo":{},"bar":{}}`, s) + suite.Require().Equal(`{"foo":{},"bar":{}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteObjectField("foo1", func() {}) @@ -76,9 +90,9 @@ func TestObject(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"foo1":{}},"bar":{"foo2":{}}}`, s) + suite.Require().Equal(`{"foo":{"foo1":{}},"bar":{"foo2":{}}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteObjectField("foo1", func() {}) @@ -89,9 +103,9 @@ func TestObject(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{}}}`, s) + suite.Require().Equal(`{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{}}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteObjectField("foo1", func() {}) @@ -103,9 +117,9 @@ func TestObject(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) + suite.Require().Equal(`{"foo":{"foo1":{},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteObjectField("foo1", func() { @@ -119,101 +133,101 @@ func TestObject(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"foo1":{"abc":null},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) + suite.Require().Equal(`{"foo":{"foo1":{"abc":null},"bar1":{}},"bar":{"foo2":{},"bar2":{}}}`, s) } -func TestBase64(t *testing.T) { +func (suite *JSONWriterTestSuite) TestBase64() { var s string - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteBase64String([]byte("foo")) }) - require.Equal(t, `"Zm9v"`, s) + suite.Require().Equal(`"Zm9v"`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteBase64StringField("foo", []byte("bar")) }) }) - require.Equal(t, `{"foo":"YmFy"}`, s) + suite.Require().Equal(`{"foo":"YmFy"}`, s) } -func TestField(t *testing.T) { +func (suite *JSONWriterTestSuite) TestField() { var s string - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteStringField("foo", "bar") }) }) - require.Equal(t, `{"foo":"bar"}`, s) + suite.Require().Equal(`{"foo":"bar"}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteUint64Field("foo", 1) }) }) - require.Equal(t, `{"foo":1}`, s) + suite.Require().Equal(`{"foo":1}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteFloat64Field("foo", 1.1) }) }) - require.Equal(t, `{"foo":1.1}`, s) + suite.Require().Equal(`{"foo":1.1}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteNullField("foo") }) }) - require.Equal(t, `{"foo":null}`, s) + suite.Require().Equal(`{"foo":null}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteAnyField("foo", nil) }) }) - require.Equal(t, `{"foo":null}`, s) + suite.Require().Equal(`{"foo":null}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteAnyField("foo", 1) }) }) - require.Equal(t, `{"foo":1}`, s) + suite.Require().Equal(`{"foo":1}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteBoolField("foo", true) }) }) - require.Equal(t, `{"foo":true}`, s) + suite.Require().Equal(`{"foo":true}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteBoolField("foo", false) }) }) - require.Equal(t, `{"foo":false}`, s) + suite.Require().Equal(`{"foo":false}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() {}) }) }) - require.Equal(t, `{"foo":{}}`, s) + suite.Require().Equal(`{"foo":{}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteUint64Field("bar", 1) }) }) }) - require.Equal(t, `{"foo":{"bar":1}}`, s) + suite.Require().Equal(`{"foo":{"bar":1}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteObjectField("foo", func() { w.WriteUint64Field("bar", 1) @@ -221,18 +235,18 @@ func TestField(t *testing.T) { }) }) }) - require.Equal(t, `{"foo":{"bar":1,"abc":"def"}}`, s) + suite.Require().Equal(`{"foo":{"bar":1,"abc":"def"}}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteUint64Field("bar", 1) w.WriteStringField("abc", "def") w.WriteNullField("xyz") }) }) - require.Equal(t, `{"bar":1,"abc":"def","xyz":null}`, s) + suite.Require().Equal(`{"bar":1,"abc":"def","xyz":null}`, s) - s = writeJSON(func(w *JSONWriter) { + s = suite.writeJSON(func(w *JSONWriter) { w.WriteObject(func() { w.WriteUint64Field("bar", 1) w.WriteStringField("abc", "def") @@ -240,5 +254,65 @@ func TestField(t *testing.T) { w.WriteObjectField("foo", func() {}) }) }) - require.Equal(t, `{"bar":1,"abc":"def","xyz":null,"foo":{}}`, s) + suite.Require().Equal(`{"bar":1,"abc":"def","xyz":null,"foo":{}}`, s) +} + +func TestExternalBuffer(t *testing.T) { + suite.Run(t, &JSONWriterTestSuite{useInternalBuffer: false}) +} + +func TestInternalBuffer(t *testing.T) { + suite.Run(t, &JSONWriterTestSuite{useInternalBuffer: true}) +} + +func BenchmarkExternalBufferWriteBase64(b *testing.B) { + out := &bytes.Buffer{} + str := randstr.Bytes(1024) + for i := 0; i < b.N; i++ { + out.Reset() + w := BorrowJSONWriter(out) + w.WriteObject(func() { + w.WriteBase64StringField("foo", str) + }) + _ = out.Bytes() + ReturnJSONWriter(w) + } +} + +func BenchmarkInternalBufferWriteBase64(b *testing.B) { + str := randstr.Bytes(1024) + for i := 0; i < b.N; i++ { + w := BorrowJSONWriter(nil) + w.WriteObject(func() { + w.WriteBase64StringField("foo", str) + }) + _ = w.Buffer() + ReturnJSONWriter(w) + } +} + +func BenchmarkExternalBufferWriteString(b *testing.B) { + out := &bytes.Buffer{} + str := randstr.String(1024) + for i := 0; i < b.N; i++ { + out.Reset() + w := BorrowJSONWriter(out) + w.WriteObject(func() { + w.WriteStringField("foo", str) + }) + _ = out.Bytes() + ReturnJSONWriter(w) + } +} + +func BenchmarkInternalBufferWriteString(b *testing.B) { + str := randstr.String(1024) + for i := 0; i < b.N; i++ { + w := BorrowJSONWriter(nil) + w.WriteObject(func() { + w.WriteStringField("foo", str) + }) + _ = w.Buffer() + ReturnJSONWriter(w) + } } From 2a1456e47a5de9b93c21965f8a015ad286ec50c3 Mon Sep 17 00:00:00 2001 From: Wish Date: Wed, 6 Dec 2023 13:44:30 +0800 Subject: [PATCH 17/25] Add example usage in code Signed-off-by: Wish --- pkg/util/json_writer.go | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go index d9e3a551cbe..365d98e65dd 100644 --- a/pkg/util/json_writer.go +++ b/pkg/util/json_writer.go @@ -32,6 +32,26 @@ var jWriterPool = sync.Pool{ }, } +// JSONWriter builds JSON in an efficient and structural way. +// +// Example Usage +// +// w := BorrowJSONWriter(out) +// +// w.WriteObject(func() { +// w.WriteObjectField("payload", func() { +// w.WriteObjectField("dml", func() { +// w.WriteStringField("statement", "INSERT") +// w.WriteUint64Field("ts", 100) +// }) +// }) +// w.WriteObjectField("source", func() { +// w.WriteStringField("source", "TiCDC") +// w.WriteInt64Field("version", 1) +// }) +// }) +// +// ReturnJSONWriter(w) type JSONWriter struct { out io.Writer stream *jsoniter.Stream // `stream` is created over `out` From 144bf987a510a34d9828996c24892032b97f5a5f Mon Sep 17 00:00:00 2001 From: Wish Date: Wed, 6 Dec 2023 13:44:30 +0800 Subject: [PATCH 18/25] Add example usage in code Signed-off-by: Wish --- pkg/util/json_writer.go | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go index d9e3a551cbe..365d98e65dd 100644 --- a/pkg/util/json_writer.go +++ b/pkg/util/json_writer.go @@ -32,6 +32,26 @@ var jWriterPool = sync.Pool{ }, } +// JSONWriter builds JSON in an efficient and structural way. +// +// Example Usage +// +// w := BorrowJSONWriter(out) +// +// w.WriteObject(func() { +// w.WriteObjectField("payload", func() { +// w.WriteObjectField("dml", func() { +// w.WriteStringField("statement", "INSERT") +// w.WriteUint64Field("ts", 100) +// }) +// }) +// w.WriteObjectField("source", func() { +// w.WriteStringField("source", "TiCDC") +// w.WriteInt64Field("version", 1) +// }) +// }) +// +// ReturnJSONWriter(w) type JSONWriter struct { out io.Writer stream *jsoniter.Stream // `stream` is created over `out` From fa5f55d444b1f7a5ceae4be31ff28c3f6c35d6d0 Mon Sep 17 00:00:00 2001 From: Wish Date: Wed, 6 Dec 2023 13:51:35 +0800 Subject: [PATCH 19/25] Add missing build dep Signed-off-by: Wish --- go.mod | 3 ++- go.sum | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 5967048505b..ca2e5e6a97f 100644 --- a/go.mod +++ b/go.mod @@ -87,6 +87,7 @@ require ( github.com/swaggo/gin-swagger v1.2.0 github.com/swaggo/swag v1.8.3 github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 + github.com/thanhpk/randstr v1.0.6 github.com/tikv/client-go/v2 v2.0.8-0.20230925032502-44b0cf7aba2b github.com/tikv/pd v1.1.0-beta.0.20230203015356-248b3f0be132 github.com/tikv/pd/client v0.0.0-20230912103610-2f57a9f050eb @@ -255,7 +256,7 @@ require ( github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df // indirect github.com/jonboulle/clockwork v0.3.0 // indirect github.com/josharian/intern v1.0.0 // indirect - github.com/json-iterator/go v1.1.12 // indirect + github.com/json-iterator/go v1.1.12 github.com/klauspost/cpuid v1.3.1 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect diff --git a/go.sum b/go.sum index fd72156f024..470848e6f14 100644 --- a/go.sum +++ b/go.sum @@ -1236,6 +1236,8 @@ github.com/swaggo/swag v1.8.3 h1:3pZSSCQ//gAH88lfmxM3Cd1+JCsxV8Md6f36b9hrZ5s= github.com/swaggo/swag v1.8.3/go.mod h1:jMLeXOOmYyjk8PvHTsXBdrubsNd9gUJTTCzL5iBnseg= github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 h1:xQdMZ1WLrgkkvOZ/LDQxjVxMLdby7osSh4ZEVa5sIjs= github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954/go.mod h1:u2MKkTVTVJWe5D1rCvame8WqhBd88EuIwODJZ1VHCPM= +github.com/thanhpk/randstr v1.0.6 h1:psAOktJFD4vV9NEVb3qkhRSMvYh4ORRaj1+w/hn4B+o= +github.com/thanhpk/randstr v1.0.6/go.mod h1:M/H2P1eNLZzlDwAzpkkkUvoyNNMbzRGhESZuEQk3r0U= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= From a762c42c5e05e18a8f839b9b37230f5aa7c6fd88 Mon Sep 17 00:00:00 2001 From: Wish Date: Wed, 6 Dec 2023 14:58:23 +0800 Subject: [PATCH 20/25] Fix lints Signed-off-by: Wish --- pkg/util/json_writer.go | 25 ++++++++++++++++++++----- pkg/util/json_writer_test.go | 16 ++++++++-------- 2 files changed, 28 insertions(+), 13 deletions(-) diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go index 365d98e65dd..c76f5514587 100644 --- a/pkg/util/json_writer.go +++ b/pkg/util/json_writer.go @@ -59,6 +59,8 @@ type JSONWriter struct { needPrependComma bool } +// BorrowJSONWriter borrows a JSONWriter instance from pool. +// Remember to call ReturnJSONWriter to return the borrowed instance. func BorrowJSONWriter(out io.Writer) *JSONWriter { w := jWriterPool.Get().(*JSONWriter) w.out = out @@ -67,6 +69,7 @@ func BorrowJSONWriter(out io.Writer) *JSONWriter { return w } +// ReturnJSONWriter returns the borrowed JSONWriter instance to pool. func ReturnJSONWriter(w *JSONWriter) { w.stream.Flush() jsonAPI.ReturnStream(w.stream) @@ -82,25 +85,27 @@ func (w *JSONWriter) Buffer() []byte { return w.stream.Buffer() } +// WriteBase64String writes a base64 string like "". func (w *JSONWriter) WriteBase64String(b []byte) { if w.out == nil { w.stream.WriteRaw(`"`) encoder := base64.NewEncoder(base64.StdEncoding, w.stream) _, _ = encoder.Write(b) - encoder.Close() + _ = encoder.Close() w.stream.WriteRaw(`"`) } else { // If out is available, let's write to out directly to avoid extra copy. // As we write to out directly so we need to flush the jsoniter stream first. - w.stream.Flush() - w.out.Write([]byte(`"`)) + _ = w.stream.Flush() + _, _ = w.out.Write([]byte(`"`)) encoder := base64.NewEncoder(base64.StdEncoding, w.out) _, _ = encoder.Write(b) - encoder.Close() - w.out.Write([]byte(`"`)) + _ = encoder.Close() + _, _ = w.out.Write([]byte(`"`)) } } +// WriteObject writes {......}. func (w *JSONWriter) WriteObject(objectFieldsWriteFn func()) { lastNeedPrependComma := w.needPrependComma w.needPrependComma = false @@ -110,6 +115,7 @@ func (w *JSONWriter) WriteObject(objectFieldsWriteFn func()) { w.needPrependComma = lastNeedPrependComma } +// WriteBoolField writes a bool field like "":. func (w *JSONWriter) WriteBoolField(fieldName string, value bool) { if w.needPrependComma { w.stream.WriteMore() @@ -120,6 +126,7 @@ func (w *JSONWriter) WriteBoolField(fieldName string, value bool) { w.stream.WriteBool(value) } +// WriteIntField writes a int field like "":. func (w *JSONWriter) WriteIntField(fieldName string, value int) { if w.needPrependComma { w.stream.WriteMore() @@ -130,6 +137,7 @@ func (w *JSONWriter) WriteIntField(fieldName string, value int) { w.stream.WriteInt(value) } +// WriteInt64Field writes a int64 field like "":. func (w *JSONWriter) WriteInt64Field(fieldName string, value int64) { if w.needPrependComma { w.stream.WriteMore() @@ -140,6 +148,7 @@ func (w *JSONWriter) WriteInt64Field(fieldName string, value int64) { w.stream.WriteInt64(value) } +// WriteUint64Field writes a uint64 field like "":. func (w *JSONWriter) WriteUint64Field(fieldName string, value uint64) { if w.needPrependComma { w.stream.WriteMore() @@ -150,6 +159,7 @@ func (w *JSONWriter) WriteUint64Field(fieldName string, value uint64) { w.stream.WriteUint64(value) } +// WriteFloat64Field writes a float64 field like "":. func (w *JSONWriter) WriteFloat64Field(fieldName string, value float64) { if w.needPrependComma { w.stream.WriteMore() @@ -160,6 +170,7 @@ func (w *JSONWriter) WriteFloat64Field(fieldName string, value float64) { w.stream.WriteFloat64(value) } +// WriteStringField writes a string field like "":"". func (w *JSONWriter) WriteStringField(fieldName string, value string) { if w.needPrependComma { w.stream.WriteMore() @@ -170,6 +181,7 @@ func (w *JSONWriter) WriteStringField(fieldName string, value string) { w.stream.WriteString(value) } +// WriteBase64StringField writes a base64 string field like "":"". func (w *JSONWriter) WriteBase64StringField(fieldName string, b []byte) { if w.needPrependComma { w.stream.WriteMore() @@ -180,6 +192,7 @@ func (w *JSONWriter) WriteBase64StringField(fieldName string, b []byte) { w.WriteBase64String(b) } +// WriteAnyField writes a field like "":. func (w *JSONWriter) WriteAnyField(fieldName string, value any) { if w.needPrependComma { w.stream.WriteMore() @@ -190,6 +203,7 @@ func (w *JSONWriter) WriteAnyField(fieldName string, value any) { w.stream.WriteVal(value) } +// WriteObjectField writes a object field like "":{......}. func (w *JSONWriter) WriteObjectField(fieldName string, objectFieldsWriteFn func()) { if w.needPrependComma { w.stream.WriteMore() @@ -200,6 +214,7 @@ func (w *JSONWriter) WriteObjectField(fieldName string, objectFieldsWriteFn func w.WriteObject(objectFieldsWriteFn) } +// WriteNullField writes a array field like "":null. func (w *JSONWriter) WriteNullField(fieldName string) { if w.needPrependComma { w.stream.WriteMore() diff --git a/pkg/util/json_writer_test.go b/pkg/util/json_writer_test.go index 11f277a0bfe..b2dfc64d4dd 100644 --- a/pkg/util/json_writer_test.go +++ b/pkg/util/json_writer_test.go @@ -26,20 +26,20 @@ type JSONWriterTestSuite struct { useInternalBuffer bool } -func (s *JSONWriterTestSuite) writeJSON(fn func(*JSONWriter)) string { - if s.useInternalBuffer { +func (suite *JSONWriterTestSuite) writeJSON(fn func(*JSONWriter)) string { + if suite.useInternalBuffer { w := BorrowJSONWriter(nil) fn(w) ret := string(w.Buffer()) ReturnJSONWriter(w) return ret - } else { - out := &bytes.Buffer{} - w := BorrowJSONWriter(out) - fn(w) - ReturnJSONWriter(w) - return out.String() } + + out := &bytes.Buffer{} + w := BorrowJSONWriter(out) + fn(w) + ReturnJSONWriter(w) + return out.String() } func (suite *JSONWriterTestSuite) TestObject() { From 4ab8e389e51ff881a57ae2bfaa381b0706c557e0 Mon Sep 17 00:00:00 2001 From: Wish Date: Fri, 8 Dec 2023 14:16:43 +0800 Subject: [PATCH 21/25] Drop integration tests. Will use another PR. Signed-off-by: Wish --- .../debezium/docker-compose.yml | 65 -- tests/integration_tests/debezium/go.mod | 84 --- tests/integration_tests/debezium/go.sum | 604 ------------------ .../debezium/sql/data_types.sql | 405 ------------ .../debezium/sql/debezium/LICENSE.txt | 202 ------ .../debezium/sql/debezium/README.txt | 3 - .../sql/debezium/binary_column_test.sql | 14 - .../sql/debezium/binary_mode_test.sql | 31 - .../debezium/sql/debezium/connector_test.sql | 73 --- .../sql/debezium/connector_test_ro.sql | 87 --- .../sql/debezium/datetime_key_test.sql | 14 - .../sql/debezium/db_default_charset.sql | 5 - .../sql/debezium/db_default_charset_noutf.sql | 6 - .../sql/debezium/decimal_column_test.sql | 14 - .../sql/debezium/enum_column_test.sql | 16 - .../sql/debezium/multitable_dbz_871.sql | 10 - .../debezium/sql/debezium/mysql_dbz_6533.sql | 18 - .../debezium/nationalized_character_test.sql | 8 - .../sql/debezium/numeric_column_test.sql | 14 - .../debezium/sql/debezium/readbinlog_test.sql | 23 - .../debezium/sql/debezium/real_test.sql | 6 - .../debezium/sql/debezium/regression_test.sql | 132 ---- .../sql/debezium/skip_messages_test.sql | 6 - .../debezium/sql/debezium/strategy_test.sql | 39 -- .../debezium/table_column_comment_test.sql | 12 - .../sql/debezium/timestamp_column_test.sql | 15 - .../debezium/sql/debezium/tinyint_test.sql | 8 - .../debezium/topic_name_sanitization_test.sql | 19 - .../sql/debezium/unsigned_integer_test.sql | 92 --- tests/integration_tests/debezium/sql/dml.sql | 24 - .../debezium/src/db_helper.go | 67 -- .../integration_tests/debezium/src/logger.go | 12 - tests/integration_tests/debezium/src/main.go | 92 --- .../debezium/src/test_cases.go | 306 --------- 34 files changed, 2526 deletions(-) delete mode 100644 tests/integration_tests/debezium/docker-compose.yml delete mode 100644 tests/integration_tests/debezium/go.mod delete mode 100644 tests/integration_tests/debezium/go.sum delete mode 100644 tests/integration_tests/debezium/sql/data_types.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/LICENSE.txt delete mode 100644 tests/integration_tests/debezium/sql/debezium/README.txt delete mode 100644 tests/integration_tests/debezium/sql/debezium/binary_column_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/connector_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/db_default_charset.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/db_default_charset_noutf.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/enum_column_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/mysql_dbz_6533.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/real_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/regression_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/skip_messages_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/strategy_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/tinyint_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql delete mode 100644 tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql delete mode 100644 tests/integration_tests/debezium/sql/dml.sql delete mode 100644 tests/integration_tests/debezium/src/db_helper.go delete mode 100644 tests/integration_tests/debezium/src/logger.go delete mode 100644 tests/integration_tests/debezium/src/main.go delete mode 100644 tests/integration_tests/debezium/src/test_cases.go diff --git a/tests/integration_tests/debezium/docker-compose.yml b/tests/integration_tests/debezium/docker-compose.yml deleted file mode 100644 index 5aa0b16384a..00000000000 --- a/tests/integration_tests/debezium/docker-compose.yml +++ /dev/null @@ -1,65 +0,0 @@ -version: "2" -services: - zookeeper: - restart: always - image: quay.io/debezium/zookeeper:2.4 - ports: - - 2181:2181 - - 2888:2888 - - 3888:3888 - kafka: - restart: always - image: quay.io/debezium/kafka:2.4 - ports: - - 9092:9092 - - 9094:9094 - depends_on: - - zookeeper - environment: - - ZOOKEEPER_CONNECT=zookeeper:2181 - - KAFKA_LISTENERS=INTERNAL://0.0.0.0:9092,OUTSIDE://0.0.0.0:9094 - - KAFKA_ADVERTISED_LISTENERS=INTERNAL://kafka:9092,OUTSIDE://localhost:9094 - - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=INTERNAL:PLAINTEXT,OUTSIDE:PLAINTEXT - - KAFKA_INTER_BROKER_LISTENER_NAME=INTERNAL - mysql: - restart: always - ports: - - 3306:3306 - image: quay.io/debezium/example-mysql:2.4 - environment: - - MYSQL_ROOT_PASSWORD= - - MYSQL_ALLOW_EMPTY_PASSWORD=yes - - MYSQL_USER=mysqluser - - MYSQL_PASSWORD=mysqlpw - connect: - restart: always - image: quay.io/debezium/connect:2.4 - ports: - - 8083:8083 - depends_on: - - kafka - - mysql - environment: - - BOOTSTRAP_SERVERS=kafka:9092 - - GROUP_ID=1 - - CONFIG_STORAGE_TOPIC=my_connect_configs - - OFFSET_STORAGE_TOPIC=my_connect_offsets - - STATUS_STORAGE_TOPIC=my_connect_statuses - watcher_dbz: # For Debug Purpose - restart: always - image: quay.io/debezium/kafka:2.4 - depends_on: - - kafka - command: watch-topic -a -k output_debezium - environment: - - ZOOKEEPER_CONNECT=zookeeper:2181 - - KAFKA_BROKER=kafka:9092 - watcher_ticdc: # For Debug Purpose - restart: always - image: quay.io/debezium/kafka:2.4 - depends_on: - - kafka - command: watch-topic -a -k output_ticdc - environment: - - ZOOKEEPER_CONNECT=zookeeper:2181 - - KAFKA_BROKER=kafka:9092 diff --git a/tests/integration_tests/debezium/go.mod b/tests/integration_tests/debezium/go.mod deleted file mode 100644 index 866df734a63..00000000000 --- a/tests/integration_tests/debezium/go.mod +++ /dev/null @@ -1,84 +0,0 @@ -module github.com/breezewish/checker - -go 1.21.0 - -require ( - github.com/go-sql-driver/mysql v1.7.1 - github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873 - github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4 - github.com/thessem/zap-prettyconsole v0.3.0 - go.uber.org/zap v1.26.0 -) - -require ( - github.com/Code-Hex/dd v1.1.0 // indirect - github.com/alecthomas/chroma v0.10.0 // indirect - github.com/beorn7/perks v1.0.1 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/cloudfoundry/gosigar v1.3.6 // indirect - github.com/cockroachdb/errors v1.8.1 // indirect - github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f // indirect - github.com/cockroachdb/redact v1.0.8 // indirect - github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 // indirect - github.com/coreos/go-semver v0.3.1 // indirect - github.com/coreos/go-systemd/v22 v22.5.0 // indirect - github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect - github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 // indirect - github.com/dlclark/regexp2 v1.4.0 // indirect - github.com/fatih/color v1.16.0 // indirect - github.com/go-ole/go-ole v1.2.6 // indirect - github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang/protobuf v1.5.3 // indirect - github.com/google/btree v1.1.2 // indirect - github.com/google/go-cmp v0.6.0 // indirect - github.com/google/uuid v1.3.1 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect - github.com/klauspost/compress v1.17.1 // indirect - github.com/kr/pretty v0.3.1 // indirect - github.com/kr/text v0.2.0 // indirect - github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect - github.com/mattn/go-colorable v0.1.13 // indirect - github.com/mattn/go-isatty v0.0.20 // indirect - github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect - github.com/opentracing/opentracing-go v1.2.0 // indirect - github.com/pierrec/lz4/v4 v4.1.15 // indirect - github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 // indirect - github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230925123611-87bebcc0d071 // indirect - github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 // indirect - github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pkg/errors v0.9.1 // indirect - github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect - github.com/prometheus/client_golang v1.17.0 // indirect - github.com/prometheus/client_model v0.5.0 // indirect - github.com/prometheus/common v0.45.0 // indirect - github.com/prometheus/procfs v0.12.0 // indirect - github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect - github.com/rogpeppe/go-internal v1.11.0 // indirect - github.com/segmentio/kafka-go v0.4.45 // indirect - github.com/shirou/gopsutil/v3 v3.23.10 // indirect - github.com/shoenig/go-m1cpu v0.1.6 // indirect - github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e // indirect - github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865 // indirect - github.com/tklauser/go-sysconf v0.3.12 // indirect - github.com/tklauser/numcpus v0.6.1 // indirect - github.com/twmb/murmur3 v1.1.6 // indirect - github.com/yusufpapurcu/wmi v1.2.3 // indirect - go.etcd.io/etcd/api/v3 v3.5.10 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect - go.etcd.io/etcd/client/v3 v3.5.10 // indirect - go.uber.org/atomic v1.11.0 // indirect - go.uber.org/multierr v1.11.0 // indirect - golang.org/x/exp v0.0.0-20231006140011-7918f672742d // indirect - golang.org/x/net v0.18.0 // indirect - golang.org/x/sync v0.4.0 // indirect - golang.org/x/sys v0.14.0 // indirect - golang.org/x/text v0.14.0 // indirect - google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/grpc v1.59.0 // indirect - google.golang.org/protobuf v1.31.0 // indirect - gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect -) diff --git a/tests/integration_tests/debezium/go.sum b/tests/integration_tests/debezium/go.sum deleted file mode 100644 index 3fe1773d286..00000000000 --- a/tests/integration_tests/debezium/go.sum +++ /dev/null @@ -1,604 +0,0 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= -github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 h1:mFRzDkZVAjdal+s7s0MwaRv9igoPqLRdzOLzw/8Xvq8= -github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358/go.mod h1:chxPXzSsl7ZWRAuOIE23GDNzjWuZquvFlgA8xmpunjU= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/toml v1.3.2 h1:o7IhLm0Msx3BaB+n3Ag7L8EVlByGnpq14C4YWiu/gL8= -github.com/BurntSushi/toml v1.3.2/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= -github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= -github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= -github.com/Code-Hex/dd v1.1.0 h1:VEtTThnS9l7WhpKUIpdcWaf0B8Vp0LeeSEsxA1DZseI= -github.com/Code-Hex/dd v1.1.0/go.mod h1:VaMyo/YjTJ3d4qm/bgtrUkT2w+aYwJ07Y7eCWyrJr1w= -github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= -github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= -github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= -github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= -github.com/alecthomas/chroma v0.10.0 h1:7XDcGkCQopCNKjZHfYrNLraA+M7e0fMiJ/Mfikbfjek= -github.com/alecthomas/chroma v0.10.0/go.mod h1:jtJATyUxlIORhUOFNA9NZDWGAQ8wpxQQqNSB4rjA/1s= -github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= -github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= -github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= -github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= -github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= -github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= -github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= -github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= -github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= -github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= -github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= -github.com/coocood/freecache v1.2.1 h1:/v1CqMq45NFH9mp/Pt142reundeBM0dVUD3osQBeu/U= -github.com/coocood/freecache v1.2.1/go.mod h1:RBUWa/Cy+OHdfTGFEhEuE1pMCMX51Ncizj7rthiQ3vk= -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/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-semver v0.3.1 h1:yi21YpKnrx1gt5R+la8n5WgS0kCrsPp33dmEyHReZr4= -github.com/coreos/go-semver v0.3.1/go.mod h1:irMmmIw/7yzSRPWryHsK7EYSg09caPQL03VsM8rvUec= -github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= -github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= -github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= -github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= -github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= -github.com/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= -github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/ristretto v0.1.1 h1:6CWw5tJNgpegArSHpNHJKldNeq03FQCwYvfMVWajOK8= -github.com/dgraph-io/ristretto v0.1.1/go.mod h1:S1GPSBCYCIhmVNfcth17y2zZtQT6wzkzgwUve0VDWWA= -github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= -github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/dlclark/regexp2 v1.4.0 h1:F1rxgk7p4uKjwIQxBs9oAXe5CqrXlCduYEJvrF4u93E= -github.com/dlclark/regexp2 v1.4.0/go.mod h1:2pZnwuY/m+8K6iRw6wQdMtk+rH5tNGR1i55kozfMjCc= -github.com/dolthub/maphash v0.1.0 h1:bsQ7JsF4FkkWyrP3oCnFJgrCUAFbFf3kOl4L/QxPDyQ= -github.com/dolthub/maphash v0.1.0/go.mod h1:gkg4Ch4CdCDu5h6PMriVLawB7koZ+5ijb9puGMV50a4= -github.com/dolthub/swiss v0.2.1 h1:gs2osYs5SJkAaH5/ggVJqXQxRXtWshF6uE0lgR/Y3Gw= -github.com/dolthub/swiss v0.2.1/go.mod h1:8AhKZZ1HK7g18j7v7k6c5cYIGEZJcPn0ARsai8cUrh0= -github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= -github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= -github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= -github.com/fatih/color v1.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM= -github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE= -github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= -github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= -github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= -github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= -github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= -github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= -github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= -github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= -github.com/go-asn1-ber/asn1-ber v1.5.4 h1:vXT6d/FNDiELJnLb6hGNa309LMsrCoYFvpwHDF0+Y1A= -github.com/go-asn1-ber/asn1-ber v1.5.4/go.mod h1:hEBeB/ic+5LoWskz+yKT7vGhhPYkProFKoKdwZRWMe0= -github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= -github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= -github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= -github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-ldap/ldap/v3 v3.4.4 h1:qPjipEpt+qDa6SI/h1fzuGWoRUY+qqQ9sOZq67/PYUs= -github.com/go-ldap/ldap/v3 v3.4.4/go.mod h1:fe1MsuN5eJJ1FeLT/LEBVdWfNWKh459R7aXgXtJC+aI= -github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= -github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= -github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= -github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrtU8EI= -github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= -github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= -github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= -github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= -github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= -github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/glog v1.1.2 h1:DVjP2PbBOzHyzA+dn3WhHIq4NdVu3Q+pvivFICf/7fo= -github.com/golang/glog v1.1.2/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= -github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= -github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= -github.com/google/btree v1.1.2 h1:xf4v41cLI2Z6FxbKm+8Bu+m8ifhj15JuZ9sa0jZCMUU= -github.com/google/btree v1.1.2/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= -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/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= -github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= -github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= -github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= -github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= -github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= -github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= -github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= -github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= -github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/influxdata/tdigest v0.0.1 h1:XpFptwYmnEKUqmkcDjrzffswZ3nvNeevbUSLPP/ZzIY= -github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= -github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= -github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= -github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= -github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= -github.com/jellydator/ttlcache/v3 v3.0.1 h1:cHgCSMS7TdQcoprXnWUptJZzyFsqs18Lt8VVhRuZYVU= -github.com/jellydator/ttlcache/v3 v3.0.1/go.mod h1:WwTaEmcXQ3MTjOm4bsZoDFiCu/hMvNWLO1w67RXz6h4= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= -github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= -github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= -github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= -github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= -github.com/kataras/iris/v12 v12.0.1/go.mod h1:udK4vLQKkdDqMGJJVd/msuMtN6hpYJhg/lSzuxjhO+U= -github.com/kataras/neffos v0.0.10/go.mod h1:ZYmJC07hQPW67eKuzlfY7SO3bC0mw83A3j6im82hfqw= -github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiDuX9AhMbDPkGYSPugBOV6yTZB1l2K9Z0= -github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= -github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.17.1 h1:NE3C767s2ak2bweCZo3+rdP4U/HoyVXLv/X9f2gPS5g= -github.com/klauspost/compress v1.17.1/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= -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/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= -github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= -github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= -github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= -github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= -github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= -github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= -github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= -github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= -github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= -github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= -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.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= -github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= -github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= -github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= -github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= -github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= -github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= -github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= -github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= -github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= -github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= -github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= -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/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= -github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= -github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= -github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= -github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= -github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= -github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= -github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= -github.com/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= -github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo= -github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= -github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= -github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= -github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= -github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= -github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= -github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= -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/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 h1:m5ZsBa5o/0CkzZXfXLaThzKuR85SnHHetqBCpzQ30h8= -github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= -github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= -github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= -github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230925123611-87bebcc0d071 h1:giqmIJSWHs+jhHfd+rth8CXWR18KAtqJu4imY1YdA6o= -github.com/pingcap/kvproto v0.0.0-20230925123611-87bebcc0d071/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= -github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= -github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= -github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= -github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873 h1:IOQPa3ItEg9RrCBgkLGO6VsKRh72LXAbdKiSfPuPQmw= -github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873/go.mod h1:yXjpyctIACAIDBLGwwRis4X8MDUO8sFS4QerAsOouAc= -github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4 h1:xrrIEHBBSeBuCL6mbyoMG4R4/ro+WpEql9XKFWtDXRw= -github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4/go.mod h1:yRkiqLFwIqibYg2P7h4bclHjHcJiIFRLKhGRyBcKYus= -github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f h1:NCiI4Wyu4GkViLGTu6cYcxt79LZ1SenBBQX1OwEV6Jg= -github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= -github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= -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/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= -github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3gMACTjAbMZBjXAqTOzOwFaj2Ld6cjeQ7Rig= -github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= -github.com/prometheus/client_golang v1.17.0 h1:rl2sfwZMtSthVU752MqfjQozy7blglC+1SOtjMAMh+Q= -github.com/prometheus/client_golang v1.17.0/go.mod h1:VeL+gMmOAxkS2IqfCq0ZmHSL+LjWfWDUmp1mBz9JgUY= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= -github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= -github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= -github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY= -github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= -github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= -github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec h1:W09IVJc94icq4NjY3clb7Lk8O1qJ8BdBEF8z0ibU0rE= -github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= -github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= -github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= -github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= -github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -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/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= -github.com/segmentio/kafka-go v0.4.45 h1:prqrZp1mMId4kI6pyPolkLsH6sWOUmDxmmucbL4WS6E= -github.com/segmentio/kafka-go v0.4.45/go.mod h1:HjF6XbOKh0Pjlkr5GVZxt6CsjjwnmhVOfURM5KMd8qg= -github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= -github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= -github.com/shirou/gopsutil/v3 v3.23.10 h1:/N42opWlYzegYaVkWejXWJpbzKv2JDy3mrgGzKsh9hM= -github.com/shirou/gopsutil/v3 v3.23.10/go.mod h1:JIE26kpucQi+innVlAUnIEOSBhBUkirr5b44yr55+WE= -github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= -github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= -github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= -github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= -github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= -github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= -github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= -github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= -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/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= -github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/thessem/zap-prettyconsole v0.3.0 h1:jreGIwOwkfqpA1NWTsbXXr0ZSL68b39KpH54q9Hz58U= -github.com/thessem/zap-prettyconsole v0.3.0/go.mod h1:93z1PhlPAYOWIOhPJvNkxE9cVM8jE5xB6/6HCtnvmcA= -github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= -github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= -github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e h1:kl8+gDOfPfRqkc1VDhhjhezMvsbfRENYsm/FqSIDnwg= -github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e/go.mod h1:fEAE7GS/lta+OasPOacdgy6RlJIRaq9/Cyr2WbSYcBE= -github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865 h1:Gkvo77EevOpBGIdV1c8gwRqPhVbgLPRy82tXNEFpGTc= -github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865/go.mod h1:cd6zBqRM9aogxf26K8NnFRPVtq9BnRE59tKEpX8IaWQ= -github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= -github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= -github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= -github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= -github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= -github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= -github.com/twmb/murmur3 v1.1.6 h1:mqrRot1BRxm+Yct+vavLMou2/iJt0tNVTTC0QoIjaZg= -github.com/twmb/murmur3 v1.1.6/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= -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 v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= -github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= -github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= -github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= -github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= -github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= -github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= -github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= -github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= -github.com/xdg-go/scram v1.1.2/go.mod h1:RT/sEzTbU5y00aCK8UOx6R7YryM0iF1N2MOmC3kKLN4= -github.com/xdg-go/stringprep v1.0.4/go.mod h1:mPGuuIYwz7CmR2bT9j4GbQqutWS1zV24gijq1dTyGkM= -github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= -github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= -github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= -github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= -github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= -github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= -github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= -github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= -github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= -github.com/yusufpapurcu/wmi v1.2.3 h1:E1ctvB7uKFMOJw3fdOW32DwGE9I7t++CRUEMKvFoFiw= -github.com/yusufpapurcu/wmi v1.2.3/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= -go.etcd.io/etcd/api/v3 v3.5.10 h1:szRajuUUbLyppkhs9K6BRtjY37l66XQQmw7oZRANE4k= -go.etcd.io/etcd/api/v3 v3.5.10/go.mod h1:TidfmT4Uycad3NM/o25fG3J07odo4GBB9hoxaodFCtI= -go.etcd.io/etcd/client/pkg/v3 v3.5.10 h1:kfYIdQftBnbAq8pUWFXfpuuxFSKzlmM5cSn76JByiT0= -go.etcd.io/etcd/client/pkg/v3 v3.5.10/go.mod h1:DYivfIviIuQ8+/lCq4vcxuseg2P2XbHygkKwFo9fc8U= -go.etcd.io/etcd/client/v3 v3.5.10 h1:W9TXNZ+oB3MCd/8UjxHTWK5J9Nquw9fQBLJd5ne5/Ao= -go.etcd.io/etcd/client/v3 v3.5.10/go.mod h1:RVeBnDz2PUEZqTpgqwAtUd8nAPf5kjyFyND7P1VkOKc= -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/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= -go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= -go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= -go.uber.org/mock v0.3.0 h1:3mUxI1No2/60yUYax92Pt8eNOEecx2D3lcXZh2NEZJo= -go.uber.org/mock v0.3.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc= -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/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= -go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= -go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= -go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= -go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= -go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= -go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= -golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= -golang.org/x/crypto v0.15.0 h1:frVn1TEaCEaZcn3Tmd7Y2b5KKPaZ+I32Q2OA3kYp5TA= -golang.org/x/crypto v0.15.0/go.mod h1:4ChreQoLWfG3xLDer1WdlH5NdlQ3+mwnQq1YTKY+72g= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231006140011-7918f672742d h1:jtJma62tbqLibJ5sFQz8bKtEM8rJBtfilJ2qTU199MI= -golang.org/x/exp v0.0.0-20231006140011-7918f672742d/go.mod h1:ldy0pHrwJyGW56pPQzzkH36rKxoZW1tw7ZJpeKx+hdo= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/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-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= -golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= -golang.org/x/net v0.18.0 h1:mIYleuAkSbHh0tCv7RvjL3F6ZVbLjq4+R7zbOn3Kokg= -golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-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/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= -golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= -golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= -golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= -golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= -golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= -golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= -golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= -golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= -golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= -golang.org/x/tools v0.14.0 h1:jvNa2pY0M4r62jkRQ6RwEZZyPcymeL9XZMLBbV7U2nc= -golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= -golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b h1:+YaDE2r2OG8t/z5qmsh7Y+XXwCbvadxxZ0YY6mTdrVA= -google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:CgAqfJo+Xmu0GwA0411Ht3OU3OntXwsGmrmjI8ioGXI= -google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b h1:CIC2YMXmIhYw6evmhPxBKJ4fmLbOFtXQN/GV3XOZR8k= -google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:IBQ646DjkDkvUIsVq/cc03FUFQ9wbZu7yE396YcL870= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b h1:ZlWIi1wSK56/8hn4QcBp/j9M7Gt3U/3hZw3mC7vDICo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc= -google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= -google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= -google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= -google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/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/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= -gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= -gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= -gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= -gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYskCTPBJVb9jqSc= -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.2.2/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.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= -gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= -gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= diff --git a/tests/integration_tests/debezium/sql/data_types.sql b/tests/integration_tests/debezium/sql/data_types.sql deleted file mode 100644 index 48452db04fe..00000000000 --- a/tests/integration_tests/debezium/sql/data_types.sql +++ /dev/null @@ -1,405 +0,0 @@ -SET sql_mode='strict_trans_tables'; -SET time_zone='+06:00'; - -/* ----------------------------------------------------------------------- --- DATE type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_date( - col DATE, - pk INT PRIMARY KEY -); - -INSERT INTO t_date VALUES ('2023-11-16', 1); -INSERT INTO t_date VALUES ('1000-01-01', 2); -INSERT INTO t_date VALUES ('9999-12-31', 3); - -SET sql_mode=''; -INSERT INTO t_date VALUES (/* Zero dates */ '0000-00-00', 4); -INSERT INTO t_date VALUES (/* Invalid dates */ '2009-11-31', 5); -SET sql_mode='strict_trans_tables'; - -/* ----------------------------------------------------------------------- --- DATETIME type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_datetime( - col DATETIME, - col_0 DATETIME(0), - col_1 DATETIME(1), - col_2 DATETIME(2), - col_3 DATETIME(3), - col_4 DATETIME(4), - col_5 DATETIME(5), - col_6 DATETIME(6), - col_z DATETIME DEFAULT 0, - -- TODO: - -- col_default_current_timestamp DATETIME DEFAULT CURRENT_TIMESTAMP, - pk INT PRIMARY KEY -); - -INSERT INTO t_datetime VALUES ( - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - NULL, - -- '2023-11-16 12:34:56.123456', - 1 -); - -INSERT INTO t_datetime VALUES ( - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - NULL, - -- '2023-11-16 12:34:56', - 2 -); - -SET time_zone='+04:00'; - -INSERT INTO t_datetime VALUES ( - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - NULL, - -- '2023-11-16 12:34:56.123456', - 3 -); - -SET time_zone='+06:00'; - -INSERT INTO t_datetime VALUES ( - NULL, - NULL, - NULL, - NULL, - NULL, - NULL, - NULL, - NULL, - NULL, - -- NULL, - 4 -); - -/* ----------------------------------------------------------------------- --- TIME type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_time( - col TIME, - col_0 TIME(0), - col_1 TIME(1), - col_5 TIME(5), - col_6 TIME(6), - pk INT PRIMARY KEY -); - -INSERT INTO t_time VALUES ( - '00:00:00', - '00:00:00', - '00:00:00', - '00:00:00', - '00:00:00', - 1 -); - -INSERT INTO t_time VALUES ( - '00:00:00.123456', - '00:00:00.123456', - '00:00:00.123456', - '00:00:00.123456', - '00:00:00.123456', - 2 -); - -INSERT INTO t_time VALUES ( - '-10:23:45.123456', - '-10:23:45.123456', - '-10:23:45.123456', - '-10:23:45.123456', - '-10:23:45.123456', - 3 -); - -/* - -Commented out because Debezium produce wrong result: - -"col":-3020399000000, -"col_0":-3020399000000, -"col_1":-3020400147483, -"col_5":-3020399048576, -"col_6":-3020399048576, - -INSERT INTO t_time VALUES ( - '-838:59:59.000000', - '-838:59:59.000000', - '-838:59:59.000000', - '-838:59:59.000000', - '-838:59:59.000000', - 3 -); - -*/ - -INSERT INTO t_time VALUES ( - '838:59:59.000000', - '838:59:59.000000', - '838:59:59.000000', - '838:59:59.000000', - '838:59:59.000000', - 4 -); - -INSERT INTO t_time VALUES ( - '123:44:56.123456', - '123:44:56.123456', - '123:44:56.123456', - '123:44:56.123456', - '123:44:56.123456', - 5 -); - -/* ----------------------------------------------------------------------- --- TIMESTAMP type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_timestamp( - col TIMESTAMP, - col_0 TIMESTAMP(0), - col_1 TIMESTAMP(1), - col_5 TIMESTAMP(5), - col_6 TIMESTAMP(6), - col_z TIMESTAMP DEFAULT 0, - pk INT PRIMARY KEY -); - -INSERT INTO t_timestamp VALUES ( - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - NULL, - 1 -); - -INSERT INTO t_timestamp VALUES ( - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - '2023-11-16 12:34:56', - NULL, - 2 -); - -SET time_zone='+04:00'; - -INSERT INTO t_timestamp VALUES ( - '2023-11-16 12:34:56.123456', - FROM_UNIXTIME(1), - FROM_UNIXTIME(1470762668), - '2023-11-16 12:34:56.123456', - '2023-11-16 12:34:56.123456', - NULL, - 3 -); - -SET time_zone='+06:00'; - -/* ----------------------------------------------------------------------- --- YEAR type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_year( - col YEAR, - col_4 YEAR(4), - pk INT PRIMARY KEY -); - -INSERT INTO t_year VALUES (1901, 1901, 1); - -/* ----------------------------------------------------------------------- --- BIT type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_bit( - col_1 BIT(1), - col_5 BIT(5), - col_6 BIT(6), - col_60 BIT(60), - pk INT PRIMARY KEY -); - -INSERT INTO t_bit VALUES (0, 16, 16, 16, 1); -INSERT INTO t_bit VALUES (1, 1, 1, 1, 2); - -/* ----------------------------------------------------------------------- --- VARCHAR type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_varchar( - col VARCHAR(64), - col_utf8_bin VARCHAR(64) - CHARACTER SET utf8mb4 COLLATE utf8mb4_bin, - /* - - Note: There is a bug of Debezium that it does not recognize - VARCHAR(..) CHARACTER SET BINARY as binary type - when the table is created AFTER the connector. - - col_bin VARCHAR(64) - CHARACTER SET BINARY, - */ - - pk INT PRIMARY KEY -); - -INSERT INTO t_varchar VALUES ('abc', 'abc', /* 'abc' , */ 1); -INSERT INTO t_varchar VALUES ('def', 'def', /* 0xAABBCC , */ 2); - -/* ----------------------------------------------------------------------- --- BINARY / VARBINARY type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_binary( - col_1 BINARY(64), - col_2 VARBINARY(64), - pk INT PRIMARY KEY -); - -INSERT INTO t_binary VALUES ('abc', 'abc', 1); -INSERT INTO t_binary VALUES ('def', 'def', 2); - -/* ----------------------------------------------------------------------- --- BLOB / TEXT type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_blob( - col_b BLOB, - col_t TEXT, - pk INT PRIMARY KEY -); - -INSERT INTO t_blob VALUES ('abc', 'abc', 1); - -/* ----------------------------------------------------------------------- --- BOOL type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_bool( - /* - - We do not support BOOL type. - Debezium supports BOOL type only when the table is created AFTER the connector. - - col_bool BOOL, - - */ - - col_tinyint_1 TINYINT(1), - col_tinyint_1_u TINYINT(1) UNSIGNED, - col_tinyint_2 TINYINT(2), - pk INT PRIMARY KEY -); - -INSERT INTO t_bool VALUES(/* true, */ 10, 10, 10, 1); -INSERT INTO t_bool VALUES(/* false, */ 10, 10, 10, 2); - - -/* ----------------------------------------------------------------------- --- FLOAT / DOUBLE type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_float( - col_f FLOAT, - col_d DOUBLE, - pk INT PRIMARY KEY -); - -INSERT INTO t_float VALUES (12345.12345, 12345.12345, 1); - -/* ----------------------------------------------------------------------- --- DECIMAL / NUMERIC type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_decimal( - col_d DECIMAL(10, 5), - col_n NUMERIC(10, 5), - pk INT PRIMARY KEY -); - -INSERT INTO t_decimal VALUES (12345.12345, 12345.12345, 1); - -/* ----------------------------------------------------------------------- --- JSON type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_json( - col JSON, - pk INT PRIMARY KEY -); - -INSERT INTO t_json VALUES ('["foo"]', 1); - -/* ----------------------------------------------------------------------- --- ENUM / SET type ----------------------------------------------------------------------- -*/ - -CREATE TABLE t_enum( - col_e ENUM('a', 'b', 'c'), - col_s SET('a', 'b', 'c'), - pk INT PRIMARY KEY -); - -INSERT INTO t_enum VALUES ('a', 'c', 1); - -SET sql_mode=''; -INSERT INTO t_enum VALUES ('d', 'e', 2); -SET sql_mode='strict_trans_tables'; diff --git a/tests/integration_tests/debezium/sql/debezium/LICENSE.txt b/tests/integration_tests/debezium/sql/debezium/LICENSE.txt deleted file mode 100644 index d6456956733..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - 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, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. diff --git a/tests/integration_tests/debezium/sql/debezium/README.txt b/tests/integration_tests/debezium/sql/debezium/README.txt deleted file mode 100644 index 6979227b498..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/README.txt +++ /dev/null @@ -1,3 +0,0 @@ -Test files in this directory is ported from: - -https://github.com/debezium/debezium/tree/main/debezium-connector-mysql/src/test/resources/ddl diff --git a/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql b/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql deleted file mode 100644 index 245452e0e27..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql +++ /dev/null @@ -1,14 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: binary_column_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE dbz_254_binary_column_test ( - id INT AUTO_INCREMENT NOT NULL, - file_uuid BINARY(16), - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8; - -INSERT INTO dbz_254_binary_column_test VALUES (default, unhex(replace('651aed08-390f-4893-b2f1-36923e7b7400','-',''))); -INSERT INTO dbz_254_binary_column_test VALUES (default, unhex(replace('651aed08-390f-4893-b2f1-36923e7b74ab','-',''))); -INSERT INTO dbz_254_binary_column_test VALUES (default, unhex(replace('651aed08-390f-4893-b2f1-36923e7b74','-',''))); -INSERT INTO dbz_254_binary_column_test VALUES (default, unhex(00)); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql b/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql deleted file mode 100644 index 8ad20ccad17..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql +++ /dev/null @@ -1,31 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: binary_column_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE dbz_1814_binary_mode_test ( - id INT AUTO_INCREMENT NOT NULL, - blob_col BLOB NOT NULL, - tinyblob_col TINYBLOB NOT NULL, - mediumblob_col MEDIUMBLOB NOT NULL, - longblob_col LONGBLOB NOT NULL, - binary_col BINARY(3) NOT NULL, - varbinary_col varbinary(20) NOT NULL, - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8; - -INSERT INTO dbz_1814_binary_mode_test ( - id, - blob_col, - tinyblob_col, - mediumblob_col, - longblob_col, - binary_col, - varbinary_col ) -VALUES ( - default, - X'010203', - X'010203', - X'010203', - X'010203', - X'010203', - X'010203' ); diff --git a/tests/integration_tests/debezium/sql/debezium/connector_test.sql b/tests/integration_tests/debezium/sql/debezium/connector_test.sql deleted file mode 100644 index 8ae03647b01..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/connector_test.sql +++ /dev/null @@ -1,73 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: connector_test --- ---------------------------------------------------------------------------------------------------------------- - --- Create and populate our products using a single insert with many rows -CREATE TABLE products ( - id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - name VARCHAR(255) NOT NULL, - description VARCHAR(512), - weight FLOAT -); -ALTER TABLE products AUTO_INCREMENT = 101; - -INSERT INTO products -VALUES (default,"scooter","Small 2-wheel scooter",3.14), - (default,"car battery","12V car battery",8.1), - (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), - (default,"hammer","12oz carpenter's hammer",0.75), - (default,"hammer","14oz carpenter's hammer",0.875), - (default,"hammer","16oz carpenter's hammer",1.0), - (default,"rocks","box of assorted rocks",5.3), - (default,"jacket","water resistent black wind breaker",0.1), - (default,"spare tire","24 inch spare tire",22.2); - --- Create and populate the products on hand using multiple inserts -CREATE TABLE products_on_hand ( - product_id INTEGER NOT NULL PRIMARY KEY, - quantity INTEGER NOT NULL, - FOREIGN KEY (product_id) REFERENCES products(id) -); - -INSERT INTO products_on_hand VALUES (101,3); -INSERT INTO products_on_hand VALUES (102,8); -INSERT INTO products_on_hand VALUES (103,18); -INSERT INTO products_on_hand VALUES (104,4); -INSERT INTO products_on_hand VALUES (105,5); -INSERT INTO products_on_hand VALUES (106,0); -INSERT INTO products_on_hand VALUES (107,44); -INSERT INTO products_on_hand VALUES (108,2); -INSERT INTO products_on_hand VALUES (109,5); - --- Create some customers ... -CREATE TABLE customers ( - id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - first_name VARCHAR(255) NOT NULL, - last_name VARCHAR(255) NOT NULL, - email VARCHAR(255) NOT NULL UNIQUE KEY -) AUTO_INCREMENT=1001; - - -INSERT INTO customers -VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), - (default,"George","Bailey","gbailey@foobar.com"), - (default,"Edward","Walker","ed@walker.com"), - (default,"Anne","Kretchmar","annek@noanswer.org"); - --- Create some very simple orders -CREATE TABLE orders ( - order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - order_date DATE NOT NULL, - purchaser INTEGER NOT NULL, - quantity INTEGER NOT NULL, - product_id INTEGER NOT NULL, - FOREIGN KEY order_customer (purchaser) REFERENCES customers(id), - FOREIGN KEY ordered_product (product_id) REFERENCES products(id) -) AUTO_INCREMENT = 10001; - -INSERT INTO orders -VALUES (default, '2016-01-16', 1001, 1, 102), - (default, '2016-01-17', 1002, 2, 105), - (default, '2016-02-18', 1004, 3, 109), - (default, '2016-02-19', 1002, 2, 106), - (default, '16-02-21', 1003, 1, 107); diff --git a/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql b/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql deleted file mode 100644 index 22ab5d1dda6..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql +++ /dev/null @@ -1,87 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: connector_test_ro --- ---------------------------------------------------------------------------------------------------------------- - --- Create and populate our products using a single insert with many rows -CREATE TABLE Products ( - PRIMARY KEY (id), - id INTEGER NOT NULL AUTO_INCREMENT, - name VARCHAR(255) NOT NULL, - description VARCHAR(512), - weight FLOAT -); -ALTER TABLE Products AUTO_INCREMENT = 101; - -INSERT INTO Products -VALUES (default,"scooter","Small 2-wheel scooter",3.14), - (default,"car battery","12V car battery",8.1), - (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), - (default,"hammer","12oz carpenter's hammer",0.75), - (default,"hammer2","14oz carpenter's hammer",8.75E-1), - (default,"hammer3","16oz carpenter's hammer",1.0), - (default,"rocks","box of assorted rocks",5.3), - (default,"jacket","water resistent black wind breaker",0.1), - (default,"spare tire","24 inch spare tire",22.2); - --- Create and populate the products on hand using multiple inserts -CREATE TABLE products_on_hand ( - product_id INTEGER NOT NULL PRIMARY KEY, - quantity INTEGER NOT NULL, - FOREIGN KEY (product_id) REFERENCES Products(id) -); - -INSERT INTO products_on_hand VALUES (101,3); -INSERT INTO products_on_hand VALUES (102,8); -INSERT INTO products_on_hand VALUES (103,18); -INSERT INTO products_on_hand VALUES (104,4); -INSERT INTO products_on_hand VALUES (105,5); -INSERT INTO products_on_hand VALUES (106,0); -INSERT INTO products_on_hand VALUES (107,44); -INSERT INTO products_on_hand VALUES (108,2); -INSERT INTO products_on_hand VALUES (109,5); - --- Create some customers ... -CREATE TABLE customers ( - id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - first_name VARCHAR(255) NOT NULL, - last_name VARCHAR(255) NOT NULL, - email VARCHAR(255) NOT NULL UNIQUE KEY -) AUTO_INCREMENT=1001; - - -INSERT INTO customers -VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), - (default,"George","Bailey","gbailey@foobar.com"), - (default,"Edward","Walker","ed@walker.com"), - (default,"Anne","Kretchmar","annek@noanswer.org"); - --- Create some very simple orders -CREATE TABLE orders ( - order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - order_date DATE NOT NULL, - purchaser INTEGER NOT NULL, - quantity INTEGER NOT NULL, - product_id INTEGER NOT NULL, - FOREIGN KEY order_customer (purchaser) REFERENCES customers(id), - FOREIGN KEY ordered_product (product_id) REFERENCES Products(id) -) AUTO_INCREMENT = 10001; - -INSERT INTO orders -VALUES (default, '2016-01-16', 1001, 1, 102), - (default, '2016-01-17', 1002, 2, 105), - (default, '2016-02-18', 1004, 3, 109), - (default, '2016-02-19', 1002, 2, 106), - (default, '2016-02-21', 1003, 1, 107); - - --- DBZ-342 handle TIME values that exceed the value range of java.sql.Time -CREATE TABLE dbz_342_timetest ( - c1 TIME(2) PRIMARY KEY, - c2 TIME(0), - c3 TIME(3), - c4 TIME(3), - c5 TIME(6) -); -INSERT INTO dbz_342_timetest VALUES ('517:51:04.777', '-13:14:50', '-733:00:00.0011', '-1:59:59.0011', '-838:59:58.999999'); - -CREATE DATABASE IF NOT EXISTS emptydb; diff --git a/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql b/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql deleted file mode 100644 index 08740721ba3..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql +++ /dev/null @@ -1,14 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: datetime_key_test --- ---------------------------------------------------------------------------------------------------------------- - -SET sql_mode=''; -CREATE TABLE dbz_1194_datetime_key_test ( - id INT AUTO_INCREMENT NOT NULL, - dtval DATETIME NOT NULL, - dval DATE NOT NULL, - tval TIME NOT NULL, - PRIMARY KEY (id, dtval, dval, tval) -) DEFAULT CHARSET=utf8; - -INSERT INTO dbz_1194_datetime_key_test VALUES (default, '0000-00-00 00:00:00', '0000-00-00', '00:00:00'); diff --git a/tests/integration_tests/debezium/sql/debezium/db_default_charset.sql b/tests/integration_tests/debezium/sql/debezium/db_default_charset.sql deleted file mode 100644 index 764da8e67e5..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/db_default_charset.sql +++ /dev/null @@ -1,5 +0,0 @@ -CREATE TABLE DATA ( - MESSAGE TEXT, - PK INT PRIMARY KEY -); -INSERT INTO DATA VALUES ('Žluťoučký', 1); diff --git a/tests/integration_tests/debezium/sql/debezium/db_default_charset_noutf.sql b/tests/integration_tests/debezium/sql/debezium/db_default_charset_noutf.sql deleted file mode 100644 index 46610614847..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/db_default_charset_noutf.sql +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE DATA ( - MESSAGE TEXT, - FLAG TINYINT(1), - PK INT PRIMARY KEY -); -INSERT INTO DATA VALUES ('Žluťoučký', 1, 1); diff --git a/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql b/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql deleted file mode 100644 index 64cd7742ad5..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql +++ /dev/null @@ -1,14 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: decimal_column_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE dbz_751_decimal_column_test ( - id INT AUTO_INCREMENT NOT NULL, - rating1 DECIMAL, - rating2 DECIMAL(8, 4), - rating3 DECIMAL(7), - rating4 DECIMAL(6, 0), - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8; - -INSERT INTO dbz_751_decimal_column_test VALUES (default, 123, 123.4567, 234.5, 345.6); diff --git a/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql b/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql deleted file mode 100644 index a8db5dfe308..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql +++ /dev/null @@ -1,16 +0,0 @@ -CREATE TABLE `test_stations_10` ( - `id` int(10) unsigned NOT NULL AUTO_INCREMENT, - `name` varchar(500) COLLATE utf8_unicode_ci NOT NULL, - `type` enum('station', 'post_office') COLLATE utf8_unicode_ci NOT NULL DEFAULT 'station', - `created` datetime DEFAULT CURRENT_TIMESTAMP, - `modified` datetime DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, - PRIMARY KEY (`id`) -); - -INSERT INTO test_stations_10 (`name`, `type`) values ( 'ha Tinh 7', 'station' ); - -ALTER TABLE `test_stations_10` - MODIFY COLUMN `type` ENUM('station', 'post_office', 'plane', 'ahihi_dongok', 'now', 'test', 'a,b', 'c,\'d', 'g,''h') - CHARACTER SET 'utf8' COLLATE 'utf8_unicode_ci' NOT NULL DEFAULT 'station'; - -INSERT INTO test_stations_10 ( `name`, `type` ) values ( 'Ha Tinh 1', 'now' ); diff --git a/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql b/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql deleted file mode 100644 index 679bb0169dc..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql +++ /dev/null @@ -1,10 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: multitable_statement --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE t1 (ID INT PRIMARY KEY); -CREATE TABLE t2 (ID INT PRIMARY KEY); -CREATE TABLE t3 (ID INT PRIMARY KEY); -CREATE TABLE t4 (ID INT PRIMARY KEY); - -DROP TABLE t1,t2,t3,t4; \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/mysql_dbz_6533.sql b/tests/integration_tests/debezium/sql/debezium/mysql_dbz_6533.sql deleted file mode 100644 index f51396ebe7f..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/mysql_dbz_6533.sql +++ /dev/null @@ -1,18 +0,0 @@ -CREATE TABLE tablename_suffix ( - PRIMARY KEY (id), - id INTEGER NOT NULL AUTO_INCREMENT -); - -CREATE TABLE tablename ( - PRIMARY KEY (id), - id INTEGER NOT NULL AUTO_INCREMENT -); - -CREATE TABLE another ( - PRIMARY KEY (id), - id INTEGER NOT NULL AUTO_INCREMENT -); - -INSERT INTO tablename_suffix VALUES (default); -INSERT INTO tablename VALUES (default); -INSERT INTO another VALUES (default); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql b/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql deleted file mode 100644 index 48cf6071c43..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE `NC_TEST` ( - `id` INT NOT NULL AUTO_INCREMENT, - `nc1` nchar default null, - `nc2` nchar(5) default null, - `nc3` nvarchar(25) default null, - PRIMARY KEY (`ID`) -) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8mb4; -INSERT INTO NC_TEST(nc1,nc2,nc3) VALUES ('a', '123', 'hello'); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql b/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql deleted file mode 100644 index 6b60b39b928..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql +++ /dev/null @@ -1,14 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: numeric_column_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE dbz_751_numeric_column_test ( - id INT AUTO_INCREMENT NOT NULL, - rating1 NUMERIC, - rating2 NUMERIC(8, 4), - rating3 NUMERIC(7), - rating4 NUMERIC(6, 0), - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8; - -INSERT INTO dbz_751_numeric_column_test VALUES (default, 123, 123.4567, 234.5, 345.6); diff --git a/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql b/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql deleted file mode 100644 index 7d6da0e272f..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql +++ /dev/null @@ -1,23 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: readbinlog_test --- Database needs to be populated to break dependency between MetadataIT and MySqlConnectorIT.shouldValidateAcceptableConfiguration run order --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE person ( - name VARCHAR(255) primary key, - birthdate DATE NULL, - age INTEGER NULL DEFAULT 10, - salary DECIMAL(5,2), bitStr BIT(18) -); -CREATE TABLE product ( - id INT NOT NULL AUTO_INCREMENT, - createdByDate DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP, - modifiedDate DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, - PRIMARY KEY(id) -); -CREATE TABLE purchased ( - purchaser VARCHAR(255) NOT NULL, - productId INT NOT NULL, - purchaseDate DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP, - PRIMARY KEY(productId,purchaser) -); diff --git a/tests/integration_tests/debezium/sql/debezium/real_test.sql b/tests/integration_tests/debezium/sql/debezium/real_test.sql deleted file mode 100644 index 7fcf64f78ff..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/real_test.sql +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE `REAL_TEST` ( - `id` INT NOT NULL AUTO_INCREMENT, - `r1` real default 1.25, - PRIMARY KEY (`ID`) -) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8mb4; -INSERT INTO REAL_TEST(r1) VALUE (2.36); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/regression_test.sql b/tests/integration_tests/debezium/sql/debezium/regression_test.sql deleted file mode 100644 index 5d396f4d44d..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/regression_test.sql +++ /dev/null @@ -1,132 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: regression_test --- ---------------------------------------------------------------------------------------------------------------- --- The integration test for this database expects to scan all of the binlog events associated with this database --- without error or problems. The integration test does not modify any records in this database, so this script --- must contain all operations to these tables. - --- DBZ-61 handle binary value recorded as hex string value -CREATE TABLE t1464075356413_testtable6 ( - pk_column int auto_increment NOT NULL, - varbinary_col varbinary(20) NOT NULL, - PRIMARY KEY(pk_column) -); -INSERT INTO t1464075356413_testtable6 (pk_column, varbinary_col) -VALUES(default, 0x4D7953514C); - --- DBZ-84 Handle TINYINT -CREATE TABLE dbz84_integer_types_table ( - -- The column lengths are used for display purposes, and do not affect the range of values - colTinyIntA tinyint NOT NULL DEFAULT 100, - colTinyIntB tinyint(1) NOT NULL DEFAULT 101, - colTinyIntC tinyint(2) UNSIGNED NOT NULL DEFAULT 102, - colTinyIntD tinyint(3) UNSIGNED NOT NULL DEFAULT 103, - colSmallIntA smallint NOT NULL DEFAULT 200, - colSmallIntB smallint(1) NOT NULL DEFAULT 201, - colSmallIntC smallint(2) NOT NULL DEFAULT 201, - colSmallIntD smallint(3) NOT NULL DEFAULT 201, - colMediumIntA mediumint NOT NULL DEFAULT 300, - colMediumIntB mediumint(1) NOT NULL DEFAULT 301, - colMediumIntC mediumint(2) NOT NULL DEFAULT 302, - colMediumIntD mediumint(3) NOT NULL DEFAULT 303, - colIntA int NOT NULL DEFAULT 400, - colIntB int(1) NOT NULL DEFAULT 401, - colIntC int(2) NOT NULL DEFAULT 402, - colIntD int(3) NOT NULL DEFAULT 403, - colBigIntA bigint NOT NULL DEFAULT 500, - colBigIntB bigint(1) NOT NULL DEFAULT 501, - colBigIntC bigint(2) NOT NULL DEFAULT 502, - colBigIntD bigint(3) NOT NULL DEFAULT 503, - PK INT PRIMARY KEY -); -INSERT INTO dbz84_integer_types_table -VALUES(127,-128,128,255, default,201,202,203, default,301,302,303, default,401,402,403, default,501,502,503, 1); - --- DBZ-85 handle fractional part of seconds -CREATE TABLE dbz_85_fractest ( - c1 DATE, - c2 TIME(2), - c3 DATETIME(2), - c4 TIMESTAMP(2), - PK INT PRIMARY KEY -); -INSERT INTO dbz_85_fractest VALUES ('2014-09-08', '17:51:04.777', '2014-09-08 17:51:04.777', '2014-09-08 17:51:04.777', 1); - --- DBZ-100 handle enum and set -CREATE TABLE dbz_100_enumsettest ( - c1 enUM('a','b','c'), - c2 Set('a','b','c'), - PK INT PRIMARY KEY -); -INSERT INTO dbz_100_enumsettest VALUES ('a', 'a,b,c', 1); -INSERT INTO dbz_100_enumsettest VALUES ('b', 'b,a', 2); -INSERT INTO dbz_100_enumsettest VALUES ('c', 'a', 3); - --- DBZ-102 handle character sets --- Use session variables to dictate the character sets used by the client running these commands so --- the literal value is interpretted correctly... -set character_set_client=utf8; -set character_set_connection=utf8; -CREATE TABLE dbz_102_charsettest ( - id INT(11) NOT NULL AUTO_INCREMENT, - text VARCHAR(255) DEFAULT NULL, - PRIMARY KEY (`id`) -) ENGINE=InnoDB AUTO_INCREMENT=2001 DEFAULT CHARSET=utf8; -INSERT INTO dbz_102_charsettest VALUES (default, "产品"); - --- DBZ-123 handle bit values, including bit field literals -CREATE TABLE dbz_123_bitvaluetest ( - c1 BIT, - c2 BIT(2), - c3 BIT(8) NOT NULL, - c4 BIT(64), - PK INT PRIMARY KEY -); -INSERT INTO dbz_123_bitvaluetest VALUES (1,2,64,23989979, 1); -INSERT INTO dbz_123_bitvaluetest VALUES (b'1',b'10',b'01000000',b'1011011100000111011011011', 2); - --- DBZ-104 handle create table like ... -DROP DATABASE IF EXISTS connector_test; -CREATE DATABASE connector_test; -CREATE TABLE connector_test.customers ( - id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - first_name VARCHAR(255) NOT NULL, - last_name VARCHAR(255) NOT NULL, - email VARCHAR(255) NOT NULL UNIQUE KEY -) AUTO_INCREMENT=1001; -INSERT INTO connector_test.customers -VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), - (default,"George","Bailey","gbailey@foobar.com"), - (default,"Edward","Walker","ed@walker.com"), - (default,"Anne","Kretchmar","annek@noanswer.org"); - --- DBZ-147 handle decimal value -CREATE TABLE dbz_147_decimalvalues ( - pk_column int auto_increment NOT NULL, - decimal_value decimal(7,2) NOT NULL, - PRIMARY KEY(pk_column) -); -INSERT INTO dbz_147_decimalvalues (pk_column, decimal_value) -VALUES(default, 12345.67); - --- DBZ-195 handle numeric values -CREATE TABLE dbz_195_numvalues ( - id int auto_increment NOT NULL, - `search_version_read` int(11) NOT NULL DEFAULT '0', -- (11) is the display width - PRIMARY KEY (`id`) -) ENGINE=InnoDB AUTO_INCREMENT=4972 DEFAULT CHARSET=utf8; - -INSERT INTO dbz_195_numvalues VALUES (default,0); -INSERT INTO dbz_195_numvalues VALUES (default,-2147483648); -INSERT INTO dbz_195_numvalues VALUES (default,2147483647); - --- DBZ-342 handle TIME values that exceed the value range of java.sql.Time -CREATE TABLE dbz_342_timetest ( - c1 TIME(2), - c2 TIME(0), - c3 TIME(3), - c4 TIME(3), - c5 TIME(6), - PK INT PRIMARY KEY -); -INSERT INTO dbz_342_timetest VALUES ('517:51:04.777', '-13:14:50', '-733:00:00.0011', '-1:59:59.0011', '-838:59:58.999999', 1); diff --git a/tests/integration_tests/debezium/sql/debezium/skip_messages_test.sql b/tests/integration_tests/debezium/sql/debezium/skip_messages_test.sql deleted file mode 100644 index b2260b71be6..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/skip_messages_test.sql +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE `debezium_test` ( - id INT NOT NULL PRIMARY KEY, - black INT, - white INT -) ENGINE=InnoDB AUTO_INCREMENT=15851 DEFAULT CHARSET=utf8; -INSERT INTO `debezium_test`(id, black, white) VALUES (0,0,0); diff --git a/tests/integration_tests/debezium/sql/debezium/strategy_test.sql b/tests/integration_tests/debezium/sql/debezium/strategy_test.sql deleted file mode 100644 index a7ba9e50f51..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/strategy_test.sql +++ /dev/null @@ -1,39 +0,0 @@ -CREATE TABLE `dbz4180` ( - id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, - a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, - b DECIMAL(10, 3) NOT NULL DEFAULT 2.321, - c VARCHAR(255) NULL DEFAULT 'default mysql strategy', - d INT NULL DEFAULT '100', - e DATETIME NULL DEFAULT CURRENT_TIMESTAMP -) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; -INSERT INTO `dbz4180`(a, b, c, d) -VALUES (1.33, -2.111 , 'topic strategy', 99); - -CREATE TABLE `dbz_4180_00` ( - id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, - a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, - b DECIMAL(10, 3) NOT NULL DEFAULT 2.321, - c VARCHAR(255) NULL DEFAULT 'shard 0', - d INT NULL DEFAULT '100', - e DATETIME NULL DEFAULT CURRENT_TIMESTAMP -) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; - -CREATE TABLE `dbz_4180_01` ( - id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, - a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, - b DECIMAL(10, 3) NOT NULL DEFAULT 2.321, - c VARCHAR(255) NULL DEFAULT 'shard 1', - d INT NULL DEFAULT '100', - e DATETIME NULL DEFAULT CURRENT_TIMESTAMP -) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; - -CREATE TABLE `dbz5743中文` ( - id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, - a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, - b DECIMAL(10, 3) NOT NULL DEFAULT 2.321, - c VARCHAR(255) NULL DEFAULT 'default mysql strategy', - d INT NULL DEFAULT '100', - e DATETIME NULL DEFAULT CURRENT_TIMESTAMP -) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; -INSERT INTO `dbz5743中文`(a, b, c, d) -VALUES (1.33, -2.111 , 'topic strategy', 99); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql b/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql deleted file mode 100644 index fc7b80aac2a..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql +++ /dev/null @@ -1,12 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: table_column_comment_test --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE dbz_4000_comment_test ( - id INT AUTO_INCREMENT NOT NULL COMMENT 'pk', - name VARCHAR(255) NOT NULL COMMENT 'this is name column', - value BIGINT NULL COMMENT 'the value is bigint type', - PRIMARY KEY (id) -) DEFAULT CHARSET=utf8 COMMENT='table for dbz-4000'; - -INSERT INTO dbz_4000_comment_test VALUES (default, 'DBZ-4000', 4000); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql b/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql deleted file mode 100644 index c80182cb59f..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql +++ /dev/null @@ -1,15 +0,0 @@ -CREATE TABLE t_user_black_list ( - `id` int(10) unsigned NOT NULL, - `data` varchar(20), - `create_time` datetime, - `update_time` datetime, - PRIMARY KEY (`id`) -); - -ALTER TABLE t_user_black_list - MODIFY COLUMN `update_time` datetime(0) NOT NULL - DEFAULT CURRENT_TIMESTAMP(0) COMMENT 'update_time' AFTER create_time; - -INSERT INTO t_user_black_list (`id`,`create_time`,`data`) VALUES (1, CURRENT_TIMESTAMP(), 'test'); - -UPDATE t_user_black_list SET `data` = 'test2' WHERE `id` = 1; \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql b/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql deleted file mode 100644 index 341dcc8f25b..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/tinyint_test.sql +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE `DBZ5236` ( - id int(11) not null primary key auto_increment, - ti1 tinyint(3) unsigned NOT NULL DEFAULT '0', - ti2 tinyint(1) unsigned NOT NULL DEFAULT '0', - ti3 tinyint(1) NOT NULL DEFAULT '1' -) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8; - -INSERT INTO DBZ5236 VALUES (DEFAULT, 1, 1, 0); diff --git a/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql b/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql deleted file mode 100644 index 878cb7cb2a2..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql +++ /dev/null @@ -1,19 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: topic-name.sanitization-it --- ---------------------------------------------------------------------------------------------------------------- - -CREATE TABLE `dbz_878_some|test@data` ( - id INT, - some_col VARCHAR(255), - PRIMARY KEY (id) - ) DEFAULT CHARSET=utf8; - -INSERT INTO `dbz_878_some|test@data` VALUES (1, 'some text'); - -CREATE TABLE `DBZ.1834` ( - id INT, - some_col VARCHAR(255), - PRIMARY KEY (id) - ) DEFAULT CHARSET=utf8; - -INSERT INTO `DBZ.1834` VALUES (1, 'some text'); diff --git a/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql b/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql deleted file mode 100644 index 1ee023a636d..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql +++ /dev/null @@ -1,92 +0,0 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: unsigned_integer_test --- ---------------------------------------------------------------------------------------------------------------- --- The integration test for this database expects to scan all of the binlog events associated with this database --- without error or problems. The integration test does not modify any records in this database, so this script --- must contain all operations to these tables. --- --- This relies upon MySQL 5.7's Geometries datatypes. - --- DBZ-228 handle unsigned TINYINT UNSIGNED -CREATE TABLE dbz_228_tinyint_unsigned ( - id int auto_increment NOT NULL, - c1 TINYINT(3) UNSIGNED ZEROFILL NOT NULL, - c2 TINYINT(3) UNSIGNED NOT NULL, - c3 TINYINT(3) NOT NULL, - PRIMARY KEY (`id`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; -INSERT INTO dbz_228_tinyint_unsigned VALUES (default, 255, 255, 127); -INSERT INTO dbz_228_tinyint_unsigned VALUES (default, 155, 155, -100); -INSERT INTO dbz_228_tinyint_unsigned VALUES (default, 0, 0, -128); - - --- DBZ-228 handle unsigned SMALLINT UNSIGNED -CREATE TABLE dbz_228_smallint_unsigned ( - id int auto_increment NOT NULL, - c1 SMALLINT UNSIGNED ZEROFILL NOT NULL, - c2 SMALLINT UNSIGNED NOT NULL, - c3 SMALLINT NOT NULL, - PRIMARY KEY (`id`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; -INSERT INTO dbz_228_smallint_unsigned VALUES (default, 65535, 65535, 32767); -INSERT INTO dbz_228_smallint_unsigned VALUES (default, 45535, 45535, -12767); -INSERT INTO dbz_228_smallint_unsigned VALUES (default, 0, 0, -32768); - - --- DBZ-228 handle unsigned MEDIUMINT UNSIGNED -CREATE TABLE dbz_228_mediumint_unsigned ( - id int auto_increment NOT NULL, - c1 MEDIUMINT UNSIGNED ZEROFILL NOT NULL, - c2 MEDIUMINT UNSIGNED NOT NULL, - c3 MEDIUMINT NOT NULL, - PRIMARY KEY (`id`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; -INSERT INTO dbz_228_mediumint_unsigned VALUES (default, 16777215, 16777215, 8388607); -INSERT INTO dbz_228_mediumint_unsigned VALUES (default, 10777215, 10777215, -6388607); -INSERT INTO dbz_228_mediumint_unsigned VALUES (default, 0, 0, -8388608); - --- DBZ-228 handle unsigned INT UNSIGNED -CREATE TABLE dbz_228_int_unsigned ( - id int auto_increment NOT NULL, - c1 int(11) UNSIGNED ZEROFILL NOT NULL, - c2 int(11) UNSIGNED NOT NULL, - c3 int(11) NOT NULL, - c4 int(5) UNSIGNED ZEROFILL NOT NULL, - c5 int(5) UNSIGNED NOT NULL , - c6 int(5) NOT NULL, - PRIMARY KEY (`id`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; -INSERT INTO dbz_228_int_unsigned VALUES (default, 4294967295, 4294967295, 2147483647, 4294967295, 4294967295, 2147483647); -INSERT INTO dbz_228_int_unsigned VALUES (default, 3294967295, 3294967295, -1147483647, 3294967295, 3294967295, -1147483647); -INSERT INTO dbz_228_int_unsigned VALUES (default, 0, 0, -2147483648, 0, 0, -2147483648); - - --- DBZ-228 handle unsigned BIGINT UNSIGNED -CREATE TABLE dbz_228_bigint_unsigned ( - id int auto_increment NOT NULL, - c1 BIGINT UNSIGNED ZEROFILL NOT NULL, - c2 BIGINT UNSIGNED NOT NULL, - c3 BIGINT NOT NULL, - PRIMARY KEY (`id`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; -INSERT INTO dbz_228_bigint_unsigned VALUES (default, 18446744073709551615, 18446744073709551615, 9223372036854775807); -INSERT INTO dbz_228_bigint_unsigned VALUES (default, 14446744073709551615, 14446744073709551615, -1223372036854775807); -INSERT INTO dbz_228_bigint_unsigned VALUES (default, 0, 0, -9223372036854775808); - --- DBZ-1185 handle SERIAL type alias -CREATE TABLE dbz_1185_serial ( - id SERIAL NOT NULL, - PRIMARY KEY (`id`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; -INSERT INTO dbz_1185_serial VALUES (10); -INSERT INTO dbz_1185_serial VALUES (default); -INSERT INTO dbz_1185_serial VALUES (18446744073709551615); - --- DBZ-1185 handle SERIAL default value -CREATE TABLE dbz_1185_serial_default_value ( - id SMALLINT UNSIGNED SERIAL DEFAULT VALUE NOT NULL, - PRIMARY KEY (`id`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; -INSERT INTO dbz_1185_serial_default_value VALUES (10); -INSERT INTO dbz_1185_serial_default_value VALUES (default); -INSERT INTO dbz_1185_serial_default_value VALUES (1000); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/dml.sql b/tests/integration_tests/debezium/sql/dml.sql deleted file mode 100644 index f8fc3f5644c..00000000000 --- a/tests/integration_tests/debezium/sql/dml.sql +++ /dev/null @@ -1,24 +0,0 @@ -CREATE TABLE foo( - PK INT PRIMARY KEY, - COL INT -); - -INSERT INTO foo VALUES (1, 1); - -INSERT INTO foo VALUES (2, 2); - -INSERT INTO foo VALUES (3, 3); - -/* Update PK */ -UPDATE foo SET PK = 5, COL = 5 WHERE COL = 3; - -/* Update Multiple Rows */ -UPDATE foo SET COL = 4; - -/* Update Single Row */ -UPDATE foo SET COL = 1 WHERE PK = 5; - -/* Update No Rows */ -UPDATE foo SET COL = 1 WHERE PK = 100; - -DELETE FROM foo WHERE PK = 3; diff --git a/tests/integration_tests/debezium/src/db_helper.go b/tests/integration_tests/debezium/src/db_helper.go deleted file mode 100644 index 25b1e3ece80..00000000000 --- a/tests/integration_tests/debezium/src/db_helper.go +++ /dev/null @@ -1,67 +0,0 @@ -package main - -import ( - "database/sql" - "strings" - - "go.uber.org/zap" -) - -type DBHelper struct { - db *sql.DB - kind Kind -} - -func NewDBHelper(kind Kind) *DBHelper { - return &DBHelper{ - db: nil, - kind: kind, - } -} - -func (h *DBHelper) MustOpen(connStringPattern string, dbName string) { - connString := strings.Replace(connStringPattern, "{db}", dbName, -1) - db, err := sql.Open("mysql", connString) - if err != nil { - logger.Panic( - "Failed to open db", - zap.String("kind", string(h.kind)), - zap.String("conn", connString), - zap.Error(err)) - } - err = db.Ping() - if err != nil { - logger.Panic( - "Failed to open db", - zap.String("kind", string(h.kind)), - zap.String("conn", connString), - zap.Error(err)) - } - h.db = db -} - -func (h *DBHelper) MustExec(query string) { - _, err := h.db.Exec(query) - if err != nil { - logger.Panic( - "Failed to execute query", - zap.String("kind", string(h.kind)), - zap.String("query", query), - zap.Error(err)) - } -} - -func (h *DBHelper) Exec(query string) error { - _, err := h.db.Exec(query) - return err -} - -func (h *DBHelper) MustClose() { - err := h.db.Close() - if err != nil { - logger.Panic( - "Failed to close connection", - zap.String("kind", string(h.kind)), - zap.Error(err)) - } -} diff --git a/tests/integration_tests/debezium/src/logger.go b/tests/integration_tests/debezium/src/logger.go deleted file mode 100644 index f899e1b6cf7..00000000000 --- a/tests/integration_tests/debezium/src/logger.go +++ /dev/null @@ -1,12 +0,0 @@ -package main - -import ( - prettyconsole "github.com/thessem/zap-prettyconsole" - "go.uber.org/zap" -) - -var logger *zap.Logger - -func init() { - logger = prettyconsole.NewLogger(zap.DebugLevel) -} diff --git a/tests/integration_tests/debezium/src/main.go b/tests/integration_tests/debezium/src/main.go deleted file mode 100644 index 5f16ba5fc7c..00000000000 --- a/tests/integration_tests/debezium/src/main.go +++ /dev/null @@ -1,92 +0,0 @@ -package main - -import ( - "flag" - "fmt" - "os" - "time" - - _ "github.com/go-sql-driver/mysql" - "github.com/google/uuid" - _ "github.com/pingcap/tidb/pkg/types/parser_driver" - "github.com/segmentio/kafka-go" - "go.uber.org/zap" -) - -var ( - dbName *string - testCaseDir *string - kafkaAddr *string -) - -var ( - dbMySQL *DBHelper - dbTiDB *DBHelper - - readerDebezium *kafka.Reader - readerTiCDC *kafka.Reader -) - -type Kind string - -const ( - KindMySQL Kind = "mysql" - KindTiDB Kind = "tidb" -) - -func prepareDBConn(kind Kind, connString string) *DBHelper { - db := NewDBHelper(kind) - db.MustOpen(connString, "") - return db -} - -func prepareKafkaConn(topic string) *kafka.Reader { - r := kafka.NewReader(kafka.ReaderConfig{ - Brokers: []string{*kafkaAddr}, - Topic: topic, - MaxBytes: 10e6, // 10MB - RetentionTime: time.Hour, - - // Config below ensures we will not read history messages. - GroupID: uuid.New().String(), - StartOffset: kafka.LastOffset, - }) - return r -} - -func buildDefaultDBConnStr(port int) string { - return fmt.Sprintf("root@tcp(127.0.0.1:%d)/{db}?allowNativePasswords=true", port) -} - -func main() { - dbConnMySQL := flag.String("db.mysql", buildDefaultDBConnStr(3306), "The connection string to connect to a MySQL instance") - dbConnTiDB := flag.String("db.tidb", buildDefaultDBConnStr(4000), "The connection string to connect to a TiDB instance") - kafkaAddr = flag.String("cdc.kafka", "127.0.0.1:9094", "") - topicDebezium := flag.String("cdc.topic.debezium", "output_debezium", "") - topicTiCDC := flag.String("cdc.topic.ticdc", "output_ticdc", "") - dbName = flag.String("db", "test", "The database to test with") - testCaseDir = flag.String("dir", "./sql", "The directory of SQL test cases") - - flag.Parse() - - logger.Info("Info", - zap.String("db.mysql", *dbConnMySQL), - zap.String("db.tidb", *dbConnTiDB), - zap.String("cdc.mysql", fmt.Sprintf("kafka://%s/%s", *kafkaAddr, *topicDebezium)), - zap.String("cdc.tidb", fmt.Sprintf("kafka://%s/%s", *kafkaAddr, *topicTiCDC)), - ) - - readerDebezium = prepareKafkaConn(*topicDebezium) - defer readerDebezium.Close() - readerTiCDC = prepareKafkaConn(*topicTiCDC) - defer readerTiCDC.Close() - - dbMySQL = prepareDBConn(KindMySQL, *dbConnMySQL) - defer dbMySQL.MustClose() - dbTiDB = prepareDBConn(KindTiDB, *dbConnTiDB) - defer dbTiDB.MustClose() - - if !runAllTestCases(*testCaseDir) { - os.Exit(1) - } -} diff --git a/tests/integration_tests/debezium/src/test_cases.go b/tests/integration_tests/debezium/src/test_cases.go deleted file mode 100644 index 208f1b970bd..00000000000 --- a/tests/integration_tests/debezium/src/test_cases.go +++ /dev/null @@ -1,306 +0,0 @@ -package main - -import ( - "context" - "encoding/json" - "errors" - "fmt" - "os" - "path/filepath" - "regexp" - "strings" - "sync" - "time" - - "github.com/alecthomas/chroma/quick" - "github.com/fatih/color" - "github.com/google/go-cmp/cmp" - "github.com/pingcap/tidb/pkg/parser" - "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/segmentio/kafka-go" - "go.uber.org/zap" -) - -var ( - nFailed = 0 - nPassed = 0 -) - -func parseSQLText(data string) (res []ast.StmtNode, warns []error, err error) { - p := parser.New() - statements, warns, err := p.Parse(data, "utf8mb4", "utf8mb4_bin") - return statements, warns, err -} - -func readAndParseSQLText(sqlFilePath string) []ast.StmtNode { - data, err := os.ReadFile(sqlFilePath) - if err != nil { - logger.Panic("Failed to read test case file", - zap.String("case", sqlFilePath), - zap.Error(err)) - } - statements, warns, err := parseSQLText(string(data)) - if warns != nil { - logger.Warn("Meet warnings when parsing SQL", - zap.String("case", sqlFilePath), - zap.Any("warnings", warns)) - } - if err != nil { - logger.Panic("Failed to parse SQL", - zap.String("case", sqlFilePath), - zap.Error(err)) - } - return statements -} - -func runAllTestCases(dir string) bool { - var files []string - err := filepath.Walk(dir, func(path string, info os.FileInfo, err error) error { - if info.IsDir() { - return nil - } - if !strings.HasSuffix(info.Name(), ".sql") { - return nil - } - files = append(files, path) - return nil - }) - if err != nil { - logger.Panic("Failed to read test case directory", zap.String("dir", dir), zap.Error(err)) - } - - for _, path := range files { - logger.Info("Run", zap.String("case", path)) - runTestCase(path) - } - - if nFailed > 0 { - logger.Error( - "Test finished with error", - zap.Int("passed", nPassed), - zap.Int("failed", nFailed)) - } else { - logger.Info( - "All tests pass", - zap.Int("passed", nPassed), - zap.Int("failed", nFailed)) - } - - return nFailed == 0 -} - -func resetDB(db *DBHelper) { - db.MustExec("drop database if exists `" + *dbName + "`;") - db.MustExec("create database `" + *dbName + "`;") - db.MustExec("use `" + *dbName + "`;") -} - -var ( - regexpExpectRows = regexp.MustCompile(`TICDC_TEST_EXPECT_ROWS:(\d+)`) -) - -func runTestCase(testCasePath string) bool { - resetDB(dbMySQL) - resetDB(dbTiDB) - - statementKindsToWaitCDCRecord := map[string]bool{ - "Delete": true, - "Insert": true, - "Replace": true, - "Update": true, - } - - hasError := false - stmtAsts := readAndParseSQLText(testCasePath) - for _, stmt := range stmtAsts { - query := strings.TrimSpace(stmt.Text()) - - waitCDCRows := false - statementKind := ast.GetStmtLabel(stmt) - if v, ok := statementKindsToWaitCDCRecord[statementKind]; v && ok { - waitCDCRows = true - } - - if runSingleQuery(query, waitCDCRows) { - nPassed++ - } else { - nFailed++ - hasError = true - } - } - - return hasError -} - -func fetchNextCDCRecord(reader *kafka.Reader, kind Kind, timeout time.Duration) (map[string]any, error) { - for { - ctx, cancel := context.WithTimeout(context.Background(), timeout) - m, err := reader.FetchMessage(ctx) - cancel() - - if err != nil { - if errors.Is(err, context.DeadlineExceeded) { - return nil, nil - } - return nil, fmt.Errorf("Failed to read CDC record of %s: %w", kind, err) - } - - if len(m.Value) == 0 { - continue - } - - var obj map[string]any - err = json.Unmarshal(m.Value, &obj) - if err != nil { - return nil, fmt.Errorf("Failed to parse CDC record of %s (msg=%s): %w", kind, m.Value, err) - } - - // Ignore DDL events in the Debezium's output - if kind == KindMySQL { - schema, ok := obj["schema"] - if !ok { - return nil, fmt.Errorf("Unexpected CDC record of %s: schema field not exist in %s", kind, m.Value) - } - if schema.(map[string]any)["name"] == "io.debezium.connector.mysql.SchemaChangeValue" { - continue - } - } - - return obj, nil - } -} - -func fetchAllCDCRecords(reader *kafka.Reader, kind Kind) []map[string]any { - var records []map[string]any - for { - waitTimeout := time.Millisecond * 500 - if len(records) == 0 { - // Wait a bit longer for the first record - if kind == KindMySQL { - waitTimeout = 3 * time.Second - } else if kind == KindTiDB { - waitTimeout = 10 * time.Second - } - } - - obj, err := fetchNextCDCRecord(reader, kind, waitTimeout) - if err != nil { - logger.Error( - "Received error when fetching CDC record", - zap.Error(err), - zap.String("kind", string(kind))) - break - } - if obj == nil { - // No more records - break - } - - records = append(records, obj) - } - - return records -} - -var ( - ignoredRecordPaths = map[string]bool{ - `{map[string]any}["schema"]`: true, - `{map[string]any}["payload"].(map[string]any)["source"]`: true, - `{map[string]any}["payload"].(map[string]any)["ts_ms"]`: true, - } -) - -var ( - headingColor = color.New(color.FgHiWhite, color.Bold) -) - -func printObj(obj any) { - v, _ := json.MarshalIndent(obj, "", " ") - quick.Highlight(os.Stdout, string(v), "json", "terminal16m", "vs") - fmt.Println() -} - -func runSingleQuery(query string, waitCDCRows bool) bool { - { - wg := &sync.WaitGroup{} - wg.Add(2) - go func() { - dbMySQL.MustExec(query) - wg.Done() - }() - go func() { - dbTiDB.MustExec(query) - wg.Done() - }() - wg.Wait() - } - - if !waitCDCRows { - return true - } - - testCasePassed := true - onError := func(err error) { - fmt.Println("==========================================") - logger.Error("Test failed", zap.Error(err)) - headingColor.Print("\nSQL:\n\n") - quick.Highlight(os.Stdout, query, "sql", "terminal16m", "vs") - fmt.Println() - testCasePassed = false - } - - var objsDebezium []map[string]any - var objsTiCDC []map[string]any - { - wg := &sync.WaitGroup{} - wg.Add(2) - go func() { - objsDebezium = fetchAllCDCRecords(readerDebezium, KindMySQL) - wg.Done() - }() - go func() { - objsTiCDC = fetchAllCDCRecords(readerTiCDC, KindTiDB) - wg.Done() - }() - wg.Wait() - } - - if len(objsDebezium) != len(objsTiCDC) { - onError(fmt.Errorf( - "Mismatch CDC rows: Got %d rows from Debezium and %d rows from TiCDC", - len(objsDebezium), - len(objsTiCDC))) - headingColor.Print("\nDebezium output:\n\n") - for _, obj := range objsDebezium { - printObj(obj) - } - headingColor.Print("\nTiCDC output:\n\n") - for _, obj := range objsTiCDC { - printObj(obj) - } - return testCasePassed - } - - cmpOption := cmp.FilterPath( - func(p cmp.Path) bool { - path := p.GoString() - _, shouldIgnore := ignoredRecordPaths[path] - return shouldIgnore - }, - cmp.Ignore(), - ) - - for i := 0; i < len(objsDebezium); i++ { - objDebezium := objsDebezium[i] - objTiCDC := objsTiCDC[i] - if diff := cmp.Diff(objDebezium, objTiCDC, cmpOption); diff != "" { - onError(fmt.Errorf("Found mismatch CDC record (output row #%d)", i+1)) - headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") - quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") - fmt.Println() - continue - } - } - - return testCasePassed -} From e76a8f91fa3c3648efeb61282dd5556836864cd1 Mon Sep 17 00:00:00 2001 From: Wish Date: Fri, 8 Dec 2023 14:37:53 +0800 Subject: [PATCH 22/25] Fix lints Signed-off-by: Wish --- pkg/sink/codec/debezium/codec.go | 61 ++++++++++++--------------- pkg/sink/codec/debezium/codec_test.go | 18 ++++---- pkg/sink/codec/debezium/encoder.go | 5 ++- 3 files changed, 40 insertions(+), 44 deletions(-) diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index 104201f1ab0..ad4d64f4e40 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -20,7 +20,6 @@ import ( "strconv" "time" - _ "github.com/goccy/go-json" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -33,13 +32,13 @@ import ( "github.com/tikv/pd/pkg/utils/tsoutil" ) -type Codec struct { +type dbzCodec struct { config *common.Config clusterID string nowFunc func() time.Time } -func (c *Codec) writeColumnsAsField(writer *util.JSONWriter, fieldName string, cols []*model.Column, colInfos []rowcodec.ColInfo) error { +func (c *dbzCodec) writeColumnsAsField(writer *util.JSONWriter, fieldName string, cols []*model.Column, colInfos []rowcodec.ColInfo) error { var err error writer.WriteObjectField(fieldName, func() { for i, col := range cols { @@ -53,7 +52,7 @@ func (c *Codec) writeColumnsAsField(writer *util.JSONWriter, fieldName string, c } // See https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-data-types -func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, ft *types.FieldType) error { +func (c *dbzCodec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, ft *types.FieldType) error { if col.Value == nil { writer.WriteNullField(col.Name) return nil @@ -69,16 +68,16 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f if n == 1 { writer.WriteBoolField(col.Name, v != 0) return nil - } else { - var buf [8]byte - binary.LittleEndian.PutUint64(buf[:], v) - numBytes := n / 8 - if n%8 != 0 { - numBytes += 1 - } - c.writeBinaryField(writer, col.Name, buf[:numBytes]) - return nil } + + var buf [8]byte + binary.LittleEndian.PutUint64(buf[:], v) + numBytes := n / 8 + if n%8 != 0 { + numBytes += 1 + } + c.writeBinaryField(writer, col.Name, buf[:numBytes]) + return nil } return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for bit column %s", @@ -95,16 +94,15 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f "unexpected column value type %T for binary string column %s", col.Value, col.Name) - } else { - if v, ok := col.Value.([]byte); ok { - writer.WriteStringField(col.Name, string(hack.String(v))) - return nil - } - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for non-binary string column %s", - col.Value, - col.Name) } + if v, ok := col.Value.([]byte); ok { + writer.WriteStringField(col.Name, string(hack.String(v))) + return nil + } + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for non-binary string column %s", + col.Value, + col.Name) case mysql.TypeEnum: if v, ok := col.Value.(uint64); ok { enumVar, err := types.ParseEnumValue(ft.GetElems(), v) @@ -159,10 +157,9 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f if mysql.HasNotNullFlag(ft.GetFlag()) { writer.WriteInt64Field(col.Name, 0) return nil - } else { - writer.WriteNullField(col.Name) - return nil } + writer.WriteNullField(col.Name) + return nil } writer.WriteInt64Field(col.Name, t.Unix()/60/60/24) return nil @@ -184,18 +181,16 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f if mysql.HasNotNullFlag(ft.GetFlag()) { writer.WriteInt64Field(col.Name, 0) return nil - } else { - writer.WriteNullField(col.Name) - return nil } + writer.WriteNullField(col.Name) + return nil } if ft.GetDecimal() <= 3 { writer.WriteInt64Field(col.Name, t.UnixMilli()) return nil - } else { - writer.WriteInt64Field(col.Name, t.UnixMicro()) - return nil } + writer.WriteInt64Field(col.Name, t.UnixMicro()) + return nil } return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for datetime column %s", @@ -281,12 +276,12 @@ func (c *Codec) writeDebeziumField(writer *util.JSONWriter, col *model.Column, f return nil } -func (c *Codec) writeBinaryField(writer *util.JSONWriter, fieldName string, value []byte) { +func (c *dbzCodec) writeBinaryField(writer *util.JSONWriter, fieldName string, value []byte) { // TODO: Deal with different binary output later. writer.WriteBase64StringField(fieldName, value) } -func (c *Codec) EncodeRowChangedEvent( +func (c *dbzCodec) EncodeRowChangedEvent( e *model.RowChangedEvent, dest io.Writer, ) error { diff --git a/pkg/sink/codec/debezium/codec_test.go b/pkg/sink/codec/debezium/codec_test.go index 64a2d72fcdf..55924ed8551 100644 --- a/pkg/sink/codec/debezium/codec_test.go +++ b/pkg/sink/codec/debezium/codec_test.go @@ -29,7 +29,7 @@ import ( ) func TestEncodeInsert(t *testing.T) { - codec := &Codec{ + codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), clusterID: "test-cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, @@ -81,11 +81,11 @@ func TestEncodeInsert(t *testing.T) { "ts_ms": 1701326309000, "transaction": null } - }`, string(buf.Bytes())) + }`, buf.String()) } func TestEncodeUpdate(t *testing.T) { - codec := &Codec{ + codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), clusterID: "test-cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, @@ -143,11 +143,11 @@ func TestEncodeUpdate(t *testing.T) { "ts_ms": 1701326309000, "transaction": null } - }`, string(buf.Bytes())) + }`, buf.String()) } func TestEncodeDelete(t *testing.T) { - codec := &Codec{ + codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), clusterID: "test-cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, @@ -199,11 +199,11 @@ func TestEncodeDelete(t *testing.T) { "ts_ms": 1701326309000, "transaction": null } - }`, string(buf.Bytes())) + }`, buf.String()) } func BenchmarkEncodeOneTinyColumn(b *testing.B) { - codec := &Codec{ + codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), clusterID: "test-cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, @@ -234,7 +234,7 @@ func BenchmarkEncodeOneTinyColumn(b *testing.B) { } func BenchmarkEncodeLargeText(b *testing.B) { - codec := &Codec{ + codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), clusterID: "test-cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, @@ -265,7 +265,7 @@ func BenchmarkEncodeLargeText(b *testing.B) { } func BenchmarkEncodeLargeBinary(b *testing.B) { - codec := &Codec{ + codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), clusterID: "test-cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go index 944d6b2a519..b5e36cb3567 100644 --- a/pkg/sink/codec/debezium/encoder.go +++ b/pkg/sink/codec/debezium/encoder.go @@ -25,11 +25,12 @@ import ( "github.com/pingcap/tiflow/pkg/sink/codec/common" ) +// BatchEncoder encodes message into Debezium format. type BatchEncoder struct { messages []*common.Message config *common.Config - codec *Codec + codec *dbzCodec } // EncodeCheckpointEvent implements the RowEventEncoder interface @@ -98,7 +99,7 @@ func newBatchEncoder(c *common.Config) codec.RowEventEncoder { batch := &BatchEncoder{ messages: nil, config: c, - codec: &Codec{ + codec: &dbzCodec{ config: c, clusterID: config.GetGlobalServerConfig().ClusterID, nowFunc: time.Now, From ba686c55764d76390e0a95aff32cebc28dd095f4 Mon Sep 17 00:00:00 2001 From: Wish Date: Mon, 11 Dec 2023 13:07:29 +0800 Subject: [PATCH 23/25] Fix tidy Signed-off-by: Wish --- go.mod | 1 - go.sum | 3 --- 2 files changed, 4 deletions(-) diff --git a/go.mod b/go.mod index fc9b7965135..5afd2a4f4b6 100644 --- a/go.mod +++ b/go.mod @@ -126,7 +126,6 @@ require ( ) require ( - github.com/AlekSi/pointer v1.2.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 // indirect github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v0.5.1 // indirect diff --git a/go.sum b/go.sum index 565f5c6ad6f..7fef763f8a0 100644 --- a/go.sum +++ b/go.sum @@ -71,8 +71,6 @@ github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 h1:/vQbFIOMb github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4/go.mod h1:hN7oaIRCjzsZ2dE+yG5k+rsdt3qcwykqK6HVGcKwsw4= github.com/99designs/keyring v1.2.1 h1:tYLp1ULvO7i3fI5vE21ReQuj99QFSs7lGm0xWyJo87o= github.com/99designs/keyring v1.2.1/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwRStKOQ5vOA= -github.com/AlekSi/pointer v1.2.0 h1:glcy/gc4h8HnG2Z3ZECSzZ1IX1x2JxRVuDzaJwQE0+w= -github.com/AlekSi/pointer v1.2.0/go.mod h1:gZGfd3dpW4vEc/UlyfKKi1roIqcCgwOIvb0tSNSBle0= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/AthenZ/athenz v1.10.39 h1:mtwHTF/v62ewY2Z5KWhuZgVXftBej1/Tn80zx4DcawY= github.com/AthenZ/athenz v1.10.39/go.mod h1:3Tg8HLsiQZp81BJY58JBeU2BR6B/H4/0MQGfCwhHNEA= @@ -591,7 +589,6 @@ github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= From 8ffca529774dc2d43b2141b1562dfb6d3207b0a3 Mon Sep 17 00:00:00 2001 From: Wish Date: Mon, 11 Dec 2023 14:28:39 +0800 Subject: [PATCH 24/25] Generate error doc Signed-off-by: Wish --- errors.toml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/errors.toml b/errors.toml index 9d5021d5de4..ac264a03eac 100755 --- a/errors.toml +++ b/errors.toml @@ -221,6 +221,11 @@ error = ''' unflatten datume data ''' +["CDC:ErrDebeziumEncodeFailed"] +error = ''' +debezium encode failed +''' + ["CDC:ErrDecodeFailed"] error = ''' decode failed: %s From f0719d5b2e240bf39068e9ab64a0b945c9f0ff3a Mon Sep 17 00:00:00 2001 From: Wish Date: Mon, 11 Dec 2023 15:53:51 +0800 Subject: [PATCH 25/25] Work with missing location files Signed-off-by: Wish --- pkg/sink/codec/debezium/debezium_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sink/codec/debezium/debezium_test.go b/pkg/sink/codec/debezium/debezium_test.go index bebec1d1312..af0da225932 100644 --- a/pkg/sink/codec/debezium/debezium_test.go +++ b/pkg/sink/codec/debezium/debezium_test.go @@ -172,7 +172,7 @@ func TestDataTypes(t *testing.T) { helper := NewSQLTestHelper(t, "foo", string(dataDDL)) helper.MustExec(`SET sql_mode='';`) - helper.MustExec(`SET time_zone='utc';`) + helper.MustExec(`SET time_zone='UTC';`) helper.MustExec(string(dataDML)) rows := helper.ScanTable()