diff --git a/Makefile b/Makefile index a64899c56..733c95443 100644 --- a/Makefile +++ b/Makefile @@ -113,7 +113,7 @@ P=3 # The following packages are used in unit tests. # Add new packages here if you want to include them in unit tests. -UT_PACKAGES_DISPATCHER := ./pkg/sink/cloudstorage/... ./pkg/sink/mysql/... ./pkg/sink/util/... ./downstreamadapter/sink/... ./downstreamadapter/dispatcher/... ./downstreamadapter/worker/... ./downstreamadapter/worker/writer/... ./pkg/sink/codec/open/... ./pkg/sink/codec/csv/... ./pkg/sink/codec/canal/... ./pkg/sink/codec/simple/... +UT_PACKAGES_DISPATCHER := ./pkg/sink/cloudstorage/... ./pkg/sink/mysql/... ./pkg/sink/util/... ./downstreamadapter/sink/... ./downstreamadapter/dispatcher/... ./downstreamadapter/worker/... ./downstreamadapter/worker/writer/... ./pkg/sink/codec/open/... ./pkg/sink/codec/csv/... ./pkg/sink/codec/canal/... ./pkg/sink/codec/debezium/... ./pkg/sink/codec/simple/... UT_PACKAGES_MAINTAINER := ./maintainer/... UT_PACKAGES_COORDINATOR := ./coordinator/... UT_PACKAGES_LOGSERVICE := ./logservice/... diff --git a/cmd/kafka-consumer/writer.go b/cmd/kafka-consumer/writer.go index 3871ba319..21931005a 100644 --- a/cmd/kafka-consumer/writer.go +++ b/cmd/kafka-consumer/writer.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tiflow/pkg/sink/codec" "github.com/pingcap/tiflow/pkg/sink/codec/avro" "github.com/pingcap/tiflow/pkg/sink/codec/canal" + "github.com/pingcap/tiflow/pkg/sink/codec/debezium" "github.com/pingcap/tiflow/pkg/sink/codec/open" "github.com/pingcap/tiflow/pkg/sink/codec/simple" "go.uber.org/zap" @@ -59,6 +60,8 @@ func NewDecoder(ctx context.Context, option *option, upstreamTiDB *sql.DB) (code decoder = avro.NewDecoder(option.codecConfig, schemaM, option.topic, upstreamTiDB) case config.ProtocolSimple: decoder, err = simple.NewDecoder(ctx, option.codecConfig, upstreamTiDB) + case config.ProtocolDebezium: + decoder = debezium.NewDecoder(option.codecConfig, upstreamTiDB) default: log.Panic("Protocol not supported", zap.Any("Protocol", option.protocol)) } @@ -471,7 +474,7 @@ func (w *writer) appendRow2Group(row *model.RowChangedEvent, progress *partition return } switch w.option.protocol { - case config.ProtocolSimple, config.ProtocolOpen, config.ProtocolCanalJSON: + case config.ProtocolSimple, config.ProtocolOpen, config.ProtocolCanalJSON, config.ProtocolDebezium: // simple protocol set the table id for all row message, it can be known which table the row message belongs to, // also consider the table partition. // open protocol set the partition table id if the table is partitioned. diff --git a/downstreamadapter/sink/kafka/sink.go b/downstreamadapter/sink/kafka/sink.go index 87ada9fa9..3deb18eb6 100644 --- a/downstreamadapter/sink/kafka/sink.go +++ b/downstreamadapter/sink/kafka/sink.go @@ -427,6 +427,12 @@ func (s *sink) sendDDLEvent(event *commonEvent.DDLEvent) error { if err != nil { return err } + if message == nil { + log.Info("Skip ddl event", zap.Uint64("commitTs", e.GetCommitTs()), + zap.String("query", e.Query), + zap.Stringer("changefeed", s.changefeedID)) + continue + } topic := s.comp.eventRouter.GetTopicForDDL(e) // Notice: We must call GetPartitionNum here, // which will be responsible for automatically creating topics when they don't exist. diff --git a/downstreamadapter/worker/cloudstorage_ddl_worker.go b/downstreamadapter/worker/cloudstorage_ddl_worker.go index 20d84eb6b..adb81f04a 100644 --- a/downstreamadapter/worker/cloudstorage_ddl_worker.go +++ b/downstreamadapter/worker/cloudstorage_ddl_worker.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/ticdc/pkg/sink/cloudstorage" "github.com/pingcap/ticdc/pkg/sink/util" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tiflow/pkg/errors" "github.com/robfig/cron" "go.uber.org/zap" @@ -83,6 +84,12 @@ func (w *CloudStorageDDLWorker) WriteBlockEvent(event *commonEvent.DDLEvent) err return err } } + if event.GetDDLType() == model.ActionExchangeTablePartition { + // For exchange partition, we need to write the schema of the source table. + var sourceTableDef cloudstorage.TableDefinition + sourceTableDef.FromTableInfo(event.ExtraSchemaName, event.ExtraTableName, event.MultipleTableInfos[1], event.GetCommitTs(), w.cfg.OutputColumnID) + return w.writeFile(event, sourceTableDef) + } event.PostFlush() return nil } diff --git a/go.mod b/go.mod index e4c3691a3..e5fb60cef 100644 --- a/go.mod +++ b/go.mod @@ -32,6 +32,7 @@ require ( github.com/golang-jwt/jwt v3.2.2+incompatible github.com/golang/mock v1.6.0 github.com/google/btree v1.1.2 + github.com/google/go-cmp v0.6.0 github.com/google/uuid v1.6.0 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/hashicorp/golang-lru v0.5.1 @@ -58,6 +59,7 @@ require ( github.com/spf13/cobra v1.8.1 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.10.0 + github.com/thanhpk/randstr v1.0.6 github.com/tikv/client-go/v2 v2.0.8-0.20241209094930-06d7f4b9233b github.com/tikv/pd v1.1.0-beta.0.20240407022249-7179657d129b github.com/tikv/pd/client v0.0.0-20241111073742-238d4d79ea31 diff --git a/logservice/schemastore/persist_storage_ddl_handlers.go b/logservice/schemastore/persist_storage_ddl_handlers.go index ea469a462..3dd3a3d0b 100644 --- a/logservice/schemastore/persist_storage_ddl_handlers.go +++ b/logservice/schemastore/persist_storage_ddl_handlers.go @@ -1334,7 +1334,11 @@ func extractTableInfoFuncForExchangeTablePartition(event *PersistedDDLEvent, tab pmodel.NewCIStr(event.TableName).O, tableID, false, - columnSchema) + columnSchema, + event.TableInfo.Charset, + event.TableInfo.Collate, + event.TableInfo.Comment, + ) return tableInfo, false } diff --git a/pkg/common/event/ddl_event.go b/pkg/common/event/ddl_event.go index 91e308469..b6880a9b3 100644 --- a/pkg/common/event/ddl_event.go +++ b/pkg/common/event/ddl_event.go @@ -146,36 +146,6 @@ func (d *DDLEvent) GetEvents() []*DDLEvent { // Some ddl event may be multi-events, we need to split it into multiple messages. // Such as rename table test.table1 to test.table10, test.table2 to test.table20 switch model.ActionType(d.Type) { - case model.ActionExchangeTablePartition: - if len(d.MultipleTableInfos) != 2 { - log.Panic("multipleTableInfos length should be equal to 2", zap.Any("multipleTableInfos", d.MultipleTableInfos)) - } - return []*DDLEvent{ - // partition table before exchange - { - Version: d.Version, - Type: d.Type, - // SchemaID: d.SchemaID, - // TableID: d.TableID, - SchemaName: d.SchemaName, - TableName: d.TableName, - TableInfo: d.MultipleTableInfos[0], - Query: d.Query, - FinishedTs: d.FinishedTs, - }, - // normal table before exchange(TODO: this may be wrong) - { - Version: d.Version, - Type: d.Type, - // SchemaID: d.TableInfo.SchemaID, - // TableID: d.TableInfo.TableName.TableID, - TableInfo: d.MultipleTableInfos[1], - SchemaName: d.ExtraSchemaName, - TableName: d.ExtraTableName, - Query: d.Query, - FinishedTs: d.FinishedTs, - }, - } case model.ActionCreateTables, model.ActionRenameTables: events := make([]*DDLEvent, 0, len(d.MultipleTableInfos)) queries, err := SplitQueries(d.Query) diff --git a/pkg/common/event/util.go b/pkg/common/event/util.go index 845e06d79..cd355e053 100644 --- a/pkg/common/event/util.go +++ b/pkg/common/event/util.go @@ -53,8 +53,8 @@ type EventTestHelper struct { tableInfos map[string]*common.TableInfo } -// NewEventTestHelper creates a SchemaTestHelper -func NewEventTestHelper(t testing.TB) *EventTestHelper { +// NewEventTestHelperWithTimeZone creates a SchemaTestHelper with time zone +func NewEventTestHelperWithTimeZone(t testing.TB, tz *time.Location) *EventTestHelper { store, err := mockstore.NewMockStore() require.NoError(t, err) ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { @@ -69,7 +69,7 @@ func NewEventTestHelper(t testing.TB) *EventTestHelper { require.NoError(t, err) - mounter := NewMounter(time.Local) + mounter := NewMounter(tz) return &EventTestHelper{ t: t, @@ -81,6 +81,11 @@ func NewEventTestHelper(t testing.TB) *EventTestHelper { } } +// NewEventTestHelper creates a SchemaTestHelper +func NewEventTestHelper(t testing.TB) *EventTestHelper { + return NewEventTestHelperWithTimeZone(t, time.Local) +} + func (s *EventTestHelper) ApplyJob(job *timodel.Job) { var tableInfo *timodel.TableInfo if job.BinlogInfo != nil && job.BinlogInfo.TableInfo != nil { diff --git a/pkg/common/table_info.go b/pkg/common/table_info.go index ecf58a483..df209bda8 100644 --- a/pkg/common/table_info.go +++ b/pkg/common/table_info.go @@ -76,6 +76,9 @@ type TableInfo struct { // record the logical ID from the DDL event(job.BinlogInfo.TableInfo). // So be careful when using the TableInfo. TableName TableName `json:"table-name"` + Charset string `json:"charset"` + Collate string `json:"collate"` + Comment string `json:"comment"` columnSchema *columnSchema `json:"-"` @@ -407,8 +410,11 @@ func newTableInfo(schema, table string, tableID int64, isPartition bool, columnS return ti } -func NewTableInfo(schemaName string, tableName string, tableID int64, isPartition bool, columnSchema *columnSchema) *TableInfo { +func NewTableInfo(schemaName string, tableName string, tableID int64, isPartition bool, columnSchema *columnSchema, charset, collate, comment string) *TableInfo { ti := newTableInfo(schemaName, tableName, tableID, isPartition, columnSchema) + ti.Charset = charset + ti.Collate = collate + ti.Comment = comment // when this tableInfo is released, we need to cut down the reference count of the columnSchema // This function should be appeared when tableInfo is created as a pair. @@ -424,8 +430,7 @@ func WrapTableInfo(schemaName string, info *model.TableInfo) *TableInfo { // search column schema object sharedColumnSchemaStorage := GetSharedColumnSchemaStorage() columnSchema := sharedColumnSchemaStorage.GetOrSetColumnSchema(info) - - return NewTableInfo(schemaName, info.Name.O, info.ID, info.GetPartitionInfo() != nil, columnSchema) + return NewTableInfo(schemaName, info.Name.O, info.ID, info.GetPartitionInfo() != nil, columnSchema, info.Charset, info.Collate, info.Comment) } // NewTableInfo4Decoder is only used by the codec decoder for the test purpose, diff --git a/pkg/errors/error.go b/pkg/errors/error.go index 6b6ab3994..b59d9cb48 100644 --- a/pkg/errors/error.go +++ b/pkg/errors/error.go @@ -54,6 +54,10 @@ var ( ) // codec related errors + ErrDDLUnsupportType = errors.Normalize( + "unsupport ddl type %s, query %s", + errors.RFCCodeText("CDC:ErrDDLUnsupportType"), + ) ErrEncodeFailed = errors.Normalize( "encode failed", errors.RFCCodeText("CDC:ErrEncodeFailed"), diff --git a/pkg/sink/codec/canal/canal_json_encoder.go b/pkg/sink/codec/canal/canal_json_encoder.go index c59e23217..dcdcab43d 100644 --- a/pkg/sink/codec/canal/canal_json_encoder.go +++ b/pkg/sink/codec/canal/canal_json_encoder.go @@ -204,8 +204,8 @@ func newJSONMessageForDML( out.String("") } - valueMap := make(map[int64]string, columnLen) // colId -> value - javaTypeMap := make(map[int64]JavaSQLType, columnLen) // colId -> javaType + valueMap := make(map[int64]string, columnLen) // colId -> value + javaTypeMap := make(map[int64]common.JavaSQLType, columnLen) // colId -> javaType row := e.GetRows() if e.IsDelete() { diff --git a/pkg/sink/codec/canal/helper.go b/pkg/sink/codec/canal/helper.go index 1f15232a9..1abc68a90 100644 --- a/pkg/sink/codec/canal/helper.go +++ b/pkg/sink/codec/canal/helper.go @@ -19,6 +19,7 @@ import ( "strconv" "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" @@ -27,16 +28,16 @@ import ( "go.uber.org/zap" ) -func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, flag uint) (string, JavaSQLType) { +func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, flag uint) (string, common.JavaSQLType) { colType := columnInfo.GetType() var ( value string - javaType JavaSQLType + javaType common.JavaSQLType ) switch colType { case mysql.TypeBit: - javaType = JavaSQLTypeBIT + javaType = common.JavaSQLTypeBIT d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -50,9 +51,9 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: bytesValue := row.GetBytes(idx) if mysql.HasBinaryFlag(flag) { - javaType = JavaSQLTypeBLOB + javaType = common.JavaSQLTypeBLOB } else { - javaType = JavaSQLTypeCLOB + javaType = common.JavaSQLTypeCLOB } if string(bytesValue) == "" { value = "null" @@ -71,9 +72,9 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl case mysql.TypeVarchar, mysql.TypeVarString: bytesValue := row.GetBytes(idx) if mysql.HasBinaryFlag(flag) { - javaType = JavaSQLTypeBLOB + javaType = common.JavaSQLTypeBLOB } else { - javaType = JavaSQLTypeVARCHAR + javaType = common.JavaSQLTypeVARCHAR } if string(bytesValue) == "" { @@ -92,9 +93,9 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl case mysql.TypeString: bytesValue := row.GetBytes(idx) if mysql.HasBinaryFlag(flag) { - javaType = JavaSQLTypeBLOB + javaType = common.JavaSQLTypeBLOB } else { - javaType = JavaSQLTypeCHAR + javaType = common.JavaSQLTypeCHAR } if string(bytesValue) == "" { value = "null" @@ -110,7 +111,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = string(bytesValue) } case mysql.TypeEnum: - javaType = JavaSQLTypeINTEGER + javaType = common.JavaSQLTypeINTEGER enumValue := row.GetEnum(idx).Value if enumValue == 0 { value = "null" @@ -118,7 +119,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = fmt.Sprintf("%d", enumValue) } case mysql.TypeSet: - javaType = JavaSQLTypeBIT + javaType = common.JavaSQLTypeBIT bitValue := row.GetEnum(idx).Value if bitValue == 0 { value = "null" @@ -126,7 +127,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = fmt.Sprintf("%d", bitValue) } case mysql.TypeDate, mysql.TypeNewDate: - javaType = JavaSQLTypeDATE + javaType = common.JavaSQLTypeDATE timeValue := row.GetTime(idx) if timeValue.IsZero() { value = "null" @@ -134,7 +135,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = timeValue.String() } case mysql.TypeDatetime, mysql.TypeTimestamp: - javaType = JavaSQLTypeTIMESTAMP + javaType = common.JavaSQLTypeTIMESTAMP timeValue := row.GetTime(idx) if timeValue.IsZero() { value = "null" @@ -142,7 +143,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = timeValue.String() } case mysql.TypeDuration: - javaType = JavaSQLTypeTIME + javaType = common.JavaSQLTypeTIME durationValue := row.GetDuration(idx, 0) if durationValue.ToNumber().IsZero() { value = "null" @@ -150,7 +151,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = durationValue.String() } case mysql.TypeJSON: - javaType = JavaSQLTypeVARCHAR + javaType = common.JavaSQLTypeVARCHAR // json needs null check before, otherwise it will panic. if row.IsNull(idx) { value = "null" @@ -163,7 +164,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl } } case mysql.TypeNewDecimal: - javaType = JavaSQLTypeDECIMAL + javaType = common.JavaSQLTypeDECIMAL decimalValue := row.GetMyDecimal(idx) if decimalValue.IsZero() { value = "null" @@ -171,7 +172,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = decimalValue.String() } case mysql.TypeInt24: - javaType = JavaSQLTypeINTEGER + javaType = common.JavaSQLTypeINTEGER d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -185,7 +186,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl } } case mysql.TypeTiny: - javaType = JavaSQLTypeTINYINT + javaType = common.JavaSQLTypeTINYINT d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -193,7 +194,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl if mysql.HasUnsignedFlag(flag) { uintValue := d.GetUint64() if uintValue > math.MaxInt8 { - javaType = JavaSQLTypeSMALLINT + javaType = common.JavaSQLTypeSMALLINT } value = strconv.FormatUint(uintValue, 10) } else { @@ -202,7 +203,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl } } case mysql.TypeShort: - javaType = JavaSQLTypeSMALLINT + javaType = common.JavaSQLTypeSMALLINT d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -210,7 +211,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl if mysql.HasUnsignedFlag(flag) { uintValue := d.GetUint64() if uintValue > math.MaxInt16 { - javaType = JavaSQLTypeINTEGER + javaType = common.JavaSQLTypeINTEGER } value = strconv.FormatUint(uintValue, 10) } else { @@ -219,7 +220,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl } } case mysql.TypeLong: - javaType = JavaSQLTypeINTEGER + javaType = common.JavaSQLTypeINTEGER d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -227,7 +228,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl if mysql.HasUnsignedFlag(flag) { uintValue := d.GetUint64() if uintValue > math.MaxInt32 { - javaType = JavaSQLTypeBIGINT + javaType = common.JavaSQLTypeBIGINT } value = strconv.FormatUint(uintValue, 10) } else { @@ -236,7 +237,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl } } case mysql.TypeLonglong: - javaType = JavaSQLTypeBIGINT + javaType = common.JavaSQLTypeBIGINT d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -244,7 +245,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl if mysql.HasUnsignedFlag(flag) { uintValue := d.GetUint64() if uintValue > math.MaxInt64 { - javaType = JavaSQLTypeDECIMAL + javaType = common.JavaSQLTypeDECIMAL } value = strconv.FormatUint(uintValue, 10) } else { @@ -253,7 +254,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl } } case mysql.TypeFloat: - javaType = JavaSQLTypeREAL + javaType = common.JavaSQLTypeREAL d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -262,7 +263,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = strconv.FormatFloat(float64(floatValue), 'f', -1, 32) } case mysql.TypeDouble: - javaType = JavaSQLTypeDOUBLE + javaType = common.JavaSQLTypeDOUBLE d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -271,7 +272,7 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = strconv.FormatFloat(floatValue, 'f', -1, 64) } case mysql.TypeYear: - javaType = JavaSQLTypeVARCHAR + javaType = common.JavaSQLTypeVARCHAR d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" @@ -280,11 +281,11 @@ func formatColumnValue(row *chunk.Row, idx int, columnInfo *model.ColumnInfo, fl value = strconv.FormatInt(yearValue, 10) } case mysql.TypeTiDBVectorFloat32: - javaType = JavaSQLTypeVARCHAR + javaType = common.JavaSQLTypeVARCHAR d := row.GetDatum(idx, &columnInfo.FieldType) value = d.GetVectorFloat32().String() default: - javaType = JavaSQLTypeVARCHAR + javaType = common.JavaSQLTypeVARCHAR d := row.GetDatum(idx, &columnInfo.FieldType) if d.IsNull() { value = "null" diff --git a/pkg/sink/codec/common/helper.go b/pkg/sink/codec/common/helper.go index 2db11872e..e64627a88 100644 --- a/pkg/sink/codec/common/helper.go +++ b/pkg/sink/codec/common/helper.go @@ -17,8 +17,10 @@ import ( "context" "database/sql" "fmt" + "math" "strconv" "strings" + "unsafe" mysqlDriver "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" @@ -373,16 +375,11 @@ func MustBinaryLiteralToInt(bytes []byte) uint64 { val, err := types.BinaryLiteral(bytes).ToInt(types.DefaultStmtNoWarningContext) if err != nil { log.Panic("invalid bit value found", zap.ByteString("value", bytes)) + return math.MaxUint64 } return val } -// FakeTableIDAllocator is a fake table id allocator -type FakeTableIDAllocator struct { - tableIDs map[string]int64 - currentTableID int64 -} - const ( replacementChar = "_" numberPrefix = 'x' @@ -466,6 +463,22 @@ func SanitizeTopicName(name string) string { return sanitizedName } +// UnsafeBytesToString create string from byte slice without copying +func UnsafeBytesToString(b []byte) string { + return *(*string)(unsafe.Pointer(&b)) +} + +// UnsafeStringToBytes create byte slice from string without copying +func UnsafeStringToBytes(s string) []byte { + return *(*[]byte)(unsafe.Pointer(&s)) +} + +// FakeTableIDAllocator is a fake table id allocator +type FakeTableIDAllocator struct { + tableIDs map[string]int64 + currentTableID int64 +} + // NewFakeTableIDAllocator creates a new FakeTableIDAllocator func NewFakeTableIDAllocator() *FakeTableIDAllocator { return &FakeTableIDAllocator{ diff --git a/pkg/sink/codec/canal/java.go b/pkg/sink/codec/common/java.go similarity index 56% rename from pkg/sink/codec/canal/java.go rename to pkg/sink/codec/common/java.go index 5ae5b642e..38d6a2d3c 100644 --- a/pkg/sink/codec/canal/java.go +++ b/pkg/sink/codec/common/java.go @@ -1,4 +1,4 @@ -// Copyright 2025 PingCAP, Inc. +// Copyright 2020 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License") // you may not use this file except in compliance with the License. @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package canal +package common import "github.com/pingcap/tidb/pkg/parser/mysql" @@ -20,31 +20,33 @@ type JavaSQLType int32 // jdk 1.8 const ( - JavaSQLTypeBIT JavaSQLType = -7 - JavaSQLTypeTINYINT JavaSQLType = -6 - JavaSQLTypeSMALLINT JavaSQLType = 5 - JavaSQLTypeINTEGER JavaSQLType = 4 - JavaSQLTypeBIGINT JavaSQLType = -5 - JavaSQLTypeREAL JavaSQLType = 7 - JavaSQLTypeDOUBLE JavaSQLType = 8 - JavaSQLTypeDECIMAL JavaSQLType = 3 - JavaSQLTypeCHAR JavaSQLType = 1 - JavaSQLTypeVARCHAR JavaSQLType = 12 - JavaSQLTypeDATE JavaSQLType = 91 - JavaSQLTypeTIME JavaSQLType = 92 - JavaSQLTypeTIMESTAMP JavaSQLType = 93 - JavaSQLTypeBINARY JavaSQLType = -2 - JavaSQLTypeVARBINARY JavaSQLType = -3 - JavaSQLTypeLONGVARBINARY JavaSQLType = -4 - JavaSQLTypeNULL JavaSQLType = 0 - JavaSQLTypeBLOB JavaSQLType = 2004 - JavaSQLTypeCLOB JavaSQLType = 2005 + JavaSQLTypeBIT JavaSQLType = -7 + JavaSQLTypeTINYINT JavaSQLType = -6 + JavaSQLTypeSMALLINT JavaSQLType = 5 + JavaSQLTypeINTEGER JavaSQLType = 4 + JavaSQLTypeBIGINT JavaSQLType = -5 + JavaSQLTypeREAL JavaSQLType = 7 + JavaSQLTypeDOUBLE JavaSQLType = 8 + JavaSQLTypeDECIMAL JavaSQLType = 3 + JavaSQLTypeCHAR JavaSQLType = 1 + JavaSQLTypeVARCHAR JavaSQLType = 12 + JavaSQLTypeDATE JavaSQLType = 91 + JavaSQLTypeTIME JavaSQLType = 92 + JavaSQLTypeTIMESTAMP JavaSQLType = 93 + JavaSQLTypeBINARY JavaSQLType = -2 + JavaSQLTypeVARBINARY JavaSQLType = -3 + JavaSQLTypeLONGVARBINARY JavaSQLType = -4 + JavaSQLTypeNULL JavaSQLType = 0 + JavaSQLTypeBLOB JavaSQLType = 2004 + JavaSQLTypeCLOB JavaSQLType = 2005 + JavaSQLTypeFLOAT JavaSQLType = 6 + JavaSQLTypeNUMERIC JavaSQLType = 2 + JavaSQLTypeOTHER JavaSQLType = 1111 + JavaSQLTypeNCHAR JavaSQLType = -15 + JavaSQLTypeTIMESTAMP_WITH_TIMEZONE JavaSQLType = 2014 //nolint // unused // JavaSQLTypeLONGVARCHAR JavaSQLType = -1 - // JavaSQLTypeFLOAT JavaSQLType = 6 - // JavaSQLTypeNUMERIC JavaSQLType = 2 - // JavaSQLTypeOTHER JavaSQLType = 1111 // JavaSQLTypeJAVA_OBJECT JavaSQLType = 2000 // JavaSQLTypeDISTINCT JavaSQLType = 2001 // JavaSQLTypeSTRUCT JavaSQLType = 2002 @@ -53,14 +55,12 @@ const ( // JavaSQLTypeDATALINK JavaSQLType = 70 // JavaSQLTypeBOOLEAN JavaSQLType = 16 // JavaSQLTypeROWID JavaSQLType = -8 - // JavaSQLTypeNCHAR JavaSQLType = -15 // JavaSQLTypeNVARCHAR JavaSQLType = -9 // JavaSQLTypeLONGNVARCHAR JavaSQLType = -16 // JavaSQLTypeNCLOB JavaSQLType = 2011 // JavaSQLTypeSQLXML JavaSQLType = 2009 // JavaSQLTypeREF_CURSOR JavaSQLType = 2012 // JavaSQLTypeTIME_WITH_TIMEZONE JavaSQLType = 2013 - // JavaSQLTypeTIMESTAMP_WITH_TIMEZONE JavaSQLType = 2014 ) // MySQLType2JavaType converts the mysql protocol types to java sql types @@ -150,3 +150,76 @@ func MySQLType2JavaType(mysqlType byte, isBinary bool) JavaSQLType { return JavaSQLTypeVARCHAR } } + +// MySQLType2JdbcType converts the mysql protocol types to jdbc type +func MySQLType2JdbcType(mysqlType byte, isBinary bool) JavaSQLType { + switch mysqlType { + case mysql.TypeTiny, mysql.TypeShort: + return JavaSQLTypeSMALLINT + + case mysql.TypeLong, mysql.TypeInt24, mysql.TypeYear: + return JavaSQLTypeINTEGER + + case mysql.TypeFloat: + return JavaSQLTypeFLOAT + + case mysql.TypeDouble: + return JavaSQLTypeDOUBLE + + case mysql.TypeNull: + return JavaSQLTypeNULL + + case mysql.TypeNewDecimal: + return JavaSQLTypeDECIMAL // equal to JavaSQLTypeNUMERIC + + case mysql.TypeTimestamp: + return JavaSQLTypeTIMESTAMP_WITH_TIMEZONE //nolint + + case mysql.TypeDatetime: + return JavaSQLTypeTIMESTAMP + + case mysql.TypeLonglong: + return JavaSQLTypeBIGINT + + case mysql.TypeDate, mysql.TypeNewDate: + return JavaSQLTypeDATE + + case mysql.TypeDuration: + return JavaSQLTypeTIME + + case mysql.TypeEnum, mysql.TypeSet: + return JavaSQLTypeCHAR + + case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + if isBinary { + return JavaSQLTypeBLOB + } + return JavaSQLTypeVARCHAR + + case mysql.TypeVarString, mysql.TypeVarchar: + if isBinary { + return JavaSQLTypeVARBINARY + } + return JavaSQLTypeVARCHAR + + case mysql.TypeString: + if isBinary { + return JavaSQLTypeBINARY + } + return JavaSQLTypeCHAR + // return JavaSQLTypeNCHAR + + // Geometry is not supported, this should not hit. + case mysql.TypeGeometry: + return JavaSQLTypeOTHER + + case mysql.TypeBit: + return JavaSQLTypeBIT + + case mysql.TypeJSON: + return JavaSQLTypeOTHER + + default: + return JavaSQLTypeVARCHAR + } +} diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index d09905a56..518087ffa 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -15,7 +15,6 @@ package debezium import ( "bytes" - "encoding/binary" "fmt" "io" "strconv" @@ -23,19 +22,21 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/common" - cerror "github.com/pingcap/ticdc/pkg/errors" - ticommon "github.com/pingcap/ticdc/pkg/sink/codec/common" + commonType "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/ticdc/pkg/util" + timodel "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util/hack" + "github.com/pingcap/tidb/pkg/util/chunk" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) type dbzCodec struct { - config *ticommon.Config + config *common.Config clusterID string nowFunc func() time.Time } @@ -43,15 +44,15 @@ type dbzCodec struct { func (c *dbzCodec) writeDebeziumFieldValues( writer *util.JSONWriter, fieldName string, - cols []*common.Column, - tableInfo *common.TableInfo, + row *chunk.Row, + tableInfo *commonType.TableInfo, ) error { var err error - colInfos := tableInfo.GetColInfosForRowChangedEvent() writer.WriteObjectField(fieldName, func() { - for i, col := range cols { - err = c.writeDebeziumFieldValue(writer, col, colInfos[i].Ft) + for i, colInfos := range tableInfo.GetColumns() { + err = c.writeDebeziumFieldValue(writer, row, i, colInfos) if err != nil { + log.Error("write Debezium field value meet error", zap.Error(err)) break } } @@ -61,20 +62,38 @@ func (c *dbzCodec) writeDebeziumFieldValues( func (c *dbzCodec) writeDebeziumFieldSchema( writer *util.JSONWriter, - col *common.Column, - ft *types.FieldType, + col *timodel.ColumnInfo, ) { - switch col.Type { - case mysql.TypeBit: - n := ft.GetFlen() - if n == 1 { - writer.WriteObjectElement(func() { + colName := col.Name.O + ft := &col.FieldType + writer.WriteObjectElement(func() { + if c.config.EnableTiDBExtension { + // The followings are TiDB extended fields + writer.WriteStringField("tidb_type", getTiDBType(ft)) + } + switch col.GetType() { + case mysql.TypeBit: + n := ft.GetFlen() + var v uint64 + var err error + if col.GetDefaultValue() != nil { + val, ok := col.GetDefaultValue().(string) + if !ok { + return + } + v, err = strconv.ParseUint(parseBit(val, n), 2, 64) + if err != nil { + return + } + } + if n == 1 { writer.WriteStringField("type", "boolean") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - } else { - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + writer.WriteBoolField("default", v != 0) // bool + } + } else { writer.WriteStringField("type", "bytes") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.data.Bits") @@ -82,32 +101,37 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteObjectField("parameters", func() { writer.WriteStringField("length", fmt.Sprintf("%d", n)) }) - writer.WriteStringField("field", col.Name) - }) - } - - case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, - mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + c.writeBinaryField(writer, "default", getBitFromUint64(n, v)) // binary + } + } + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, + mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: writer.WriteStringField("type", "string") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeEnum: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } + case mysql.TypeEnum: writer.WriteStringField("type", "string") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.data.Enum") writer.WriteIntField("version", 1) writer.WriteObjectField("parameters", func() { - writer.WriteStringField("allowed", strings.Join(ft.GetElems(), ",")) + elems := ft.GetElems() + parameters := make([]string, 0, len(elems)) + for _, ele := range elems { + parameters = append(parameters, common.EscapeEnumAndSetOptions(ele)) + } + writer.WriteStringField("allowed", strings.Join(parameters, ",")) }) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeSet: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } + case mysql.TypeSet: writer.WriteStringField("type", "string") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.data.EnumSet") @@ -115,27 +139,41 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteObjectField("parameters", func() { writer.WriteStringField("allowed", strings.Join(ft.GetElems(), ",")) }) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeNewDecimal: - writer.WriteObjectElement(func() { - writer.WriteStringField("type", "double") - writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeDate, mysql.TypeNewDate: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } + case mysql.TypeDate, mysql.TypeNewDate: writer.WriteStringField("type", "int32") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.time.Date") writer.WriteIntField("version", 1) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeDatetime: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + 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()) { + writer.WriteInt64Field("default", 0) + } + return + } + year := t.Year() + if year < 70 { + // treats "0018" as 2018 + t = t.AddDate(2000, 0, 0) + } else if year < 100 { + // treats "0099" as 1999 + t = t.AddDate(1900, 0, 0) + } + writer.WriteInt64Field("default", t.UTC().Unix()/60/60/24) + } + case mysql.TypeDatetime: writer.WriteStringField("type", "int64") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) if ft.GetDecimal() <= 3 { @@ -144,116 +182,269 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("name", "io.debezium.time.MicroTimestamp") } writer.WriteIntField("version", 1) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeTimestamp: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + if v == "CURRENT_TIMESTAMP" { + writer.WriteInt64Field("default", 0) + return + } + t, err := types.StrToDateTime(types.DefaultStmtNoWarningContext, v, ft.GetDecimal()) + if err != nil { + writer.WriteInt64Field("default", 0) + return + } + gt, err := t.GoTime(time.UTC) + if err != nil { + if mysql.HasNotNullFlag(ft.GetFlag()) { + writer.WriteInt64Field("default", 0) + } + return + } + year := gt.Year() + if year < 70 { + // treats "0018" as 2018 + gt = gt.AddDate(2000, 0, 0) + } else if year < 100 { + // treats "0099" as 1999 + gt = gt.AddDate(1900, 0, 0) + } + if ft.GetDecimal() <= 3 { + writer.WriteInt64Field("default", gt.UnixMilli()) + } else { + writer.WriteInt64Field("default", gt.UnixMicro()) + } + } + case mysql.TypeTimestamp: writer.WriteStringField("type", "string") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.time.ZonedTimestamp") writer.WriteIntField("version", 1) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeDuration: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + if v == "CURRENT_TIMESTAMP" { + if mysql.HasNotNullFlag(ft.GetFlag()) { + writer.WriteStringField("default", "1970-01-01T00:00:00Z") + } + return + } + t, err := types.StrToDateTime(types.DefaultStmtNoWarningContext, v, ft.GetDecimal()) + if err != nil { + writer.WriteInt64Field("default", 0) + return + } + if t.Compare(types.MinTimestamp) < 0 { + if mysql.HasNotNullFlag(ft.GetFlag()) { + writer.WriteStringField("default", "1970-01-01T00:00:00Z") + } + return + } + gt, err := t.GoTime(time.UTC) + if err != nil { + writer.WriteInt64Field("default", 0) + return + } + str := gt.Format("2006-01-02T15:04:05") + fsp := ft.GetDecimal() + if fsp > 0 { + tmp := fmt.Sprintf(".%06d", gt.Nanosecond()/1000) + str = str + tmp[:1+fsp] + } + str += "Z" + writer.WriteStringField("default", str) + } + case mysql.TypeDuration: writer.WriteStringField("type", "int64") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.time.MicroTime") writer.WriteIntField("version", 1) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeJSON: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + d, _, _, err := types.StrToDuration(types.DefaultStmtNoWarningContext.WithLocation(c.config.TimeZone), v, ft.GetDecimal()) + if err != nil { + return + } + writer.WriteInt64Field("default", d.Microseconds()) + } + case mysql.TypeJSON: writer.WriteStringField("type", "string") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.data.Json") writer.WriteIntField("version", 1) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeTiny: // TINYINT - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } + case mysql.TypeTiny: // TINYINT writer.WriteStringField("type", "int16") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeShort: // SMALLINT - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } + case mysql.TypeShort: // SMALLINT if mysql.HasUnsignedFlag(ft.GetFlag()) { writer.WriteStringField("type", "int32") } else { writer.WriteStringField("type", "int16") } writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeInt24: // MEDIUMINT - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } + case mysql.TypeInt24: // MEDIUMINT writer.WriteStringField("type", "int32") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeLong: // INT - writer.WriteObjectElement(func() { - if mysql.HasUnsignedFlag(ft.GetFlag()) { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } + case mysql.TypeLong: // INT + if mysql.HasUnsignedFlag(col.GetFlag()) { writer.WriteStringField("type", "int64") } else { writer.WriteStringField("type", "int32") } writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeLonglong: // BIGINT - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } + case mysql.TypeLonglong: // BIGINT writer.WriteStringField("type", "int64") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeFloat: - writer.WriteObjectElement(func() { - writer.WriteStringField("type", "float") + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } + case mysql.TypeFloat: + if ft.GetDecimal() != -1 { + writer.WriteStringField("type", "double") + } else { + writer.WriteStringField("type", "float") + } writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeDouble: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } + case mysql.TypeDouble, mysql.TypeNewDecimal: + // https://dev.mysql.com/doc/refman/8.4/en/numeric-types.html + // MySQL also treats REAL as a synonym for DOUBLE PRECISION (a nonstandard variation), unless the REAL_AS_FLOAT SQL mode is enabled. writer.WriteStringField("type", "double") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.Name) - }) - - case mysql.TypeYear: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } + case mysql.TypeYear: writer.WriteStringField("type", "int32") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.time.Year") writer.WriteIntField("version", 1) - writer.WriteStringField("field", col.Name) - }) - case mysql.TypeTiDBVectorFloat32: - writer.WriteObjectElement(func() { + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + if floatV < 70 { + // treats "DEFAULT 1" as 2001 + floatV += 2000 + } else if floatV < 100 { + // treats "DEFAULT 99" as 1999 + floatV += 1900 + } + writer.WriteFloat64Field("default", floatV) + } + case mysql.TypeTiDBVectorFloat32: writer.WriteStringField("type", "string") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.data.TiDBVectorFloat32") - writer.WriteIntField("version", 1) - writer.WriteStringField("field", col.Name) - }) - default: - log.Warn( - "meet unsupported field type", - zap.Any("fieldType", col.Type), - zap.Any("column", col.Name), - ) - } + writer.WriteStringField("field", colName) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } + default: + log.Warn( + "meet unsupported field type", + zap.Any("fieldType", col.GetType()), + zap.Any("column", colName), + ) + } + }) } // See https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-data-types @@ -261,175 +452,160 @@ func (c *dbzCodec) writeDebeziumFieldSchema( //revive:disable indent-error-flow func (c *dbzCodec) writeDebeziumFieldValue( writer *util.JSONWriter, - col *common.Column, - ft *types.FieldType, + row *chunk.Row, + idx int, + colInfo *timodel.ColumnInfo, ) error { - if col.Value == nil { - writer.WriteNullField(col.Name) - return nil + ft := &colInfo.FieldType + colName := colInfo.Name.O + datum := row.GetDatum(idx, ft) + if datum.IsNull() { + defaultVal := colInfo.GetDefaultValue() + if defaultVal == nil { + writer.WriteNullField(colName) + return nil + } + val := getValueFromDefault(defaultVal, ft) + datum.SetValue(val, ft) } - switch col.Type { + switch colInfo.GetType() { case mysql.TypeBit: - v, ok := col.Value.(uint64) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for bit column %s", - col.Value, - col.Name) + n := ft.GetFlen() + v, err := datum.GetMysqlBit().ToInt(types.DefaultStmtNoWarningContext) + if err != nil { + return errors.WrapError( + errors.ErrDebeziumEncodeFailed, + err) } - // 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 { - writer.WriteBoolField(col.Name, v != 0) + writer.WriteBoolField(colName, 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]) + c.writeBinaryField(writer, colName, getBitFromUint64(n, v)) return nil } case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: - v, ok := col.Value.([]byte) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for string column %s", - col.Value, - col.Name) - } - - if col.Flag.IsBinary() { - c.writeBinaryField(writer, col.Name, v) - return nil + v := datum.GetBytes() + isBinary := mysql.HasBinaryFlag(colInfo.GetFlag()) + if !isBinary { + writer.WriteStringField(colName, common.UnsafeBytesToString(v)) } else { - writer.WriteStringField(col.Name, string(hack.String(v))) - return nil + c.writeBinaryField(writer, colName, v) } + return nil case mysql.TypeEnum: - v, ok := col.Value.(uint64) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for enum column %s", - col.Value, - col.Name) - } - + v := datum.GetMysqlEnum().Value enumVar, err := types.ParseEnumValue(ft.GetElems(), v) if err != nil { // Invalid enum value inserted in non-strict mode. - writer.WriteStringField(col.Name, "") + writer.WriteStringField(colName, "") return nil } - - writer.WriteStringField(col.Name, enumVar.Name) + writer.WriteStringField(colName, enumVar.Name) return nil case mysql.TypeSet: - v, ok := col.Value.(uint64) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for set column %s", - col.Value, - col.Name) - } - + v := datum.GetMysqlSet().Value setVar, err := types.ParseSetValue(ft.GetElems(), v) if err != nil { // Invalid enum value inserted in non-strict mode. - writer.WriteStringField(col.Name, "") + writer.WriteStringField(colName, "") return nil } - - writer.WriteStringField(col.Name, setVar.Name) + writer.WriteStringField(colName, setVar.Name) return nil case mysql.TypeNewDecimal: - v, ok := col.Value.(string) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for decimal column %s", - col.Value, - col.Name) - } - - floatV, err := strconv.ParseFloat(v, 64) + v, err := datum.GetMysqlDecimal().ToFloat64() if err != nil { - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, + return errors.WrapError( + errors.ErrDebeziumEncodeFailed, err) } - - writer.WriteFloat64Field(col.Name, floatV) + writer.WriteFloat64Field(colName, v) return nil case mysql.TypeDate, mysql.TypeNewDate: - v, ok := col.Value.(string) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for date column %s", - col.Value, - col.Name) - } - + v := datum.GetMysqlTime().String() 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()) { - writer.WriteInt64Field(col.Name, 0) - return nil + writer.WriteInt64Field(colName, 0) } else { - writer.WriteNullField(col.Name) - return nil + writer.WriteNullField(colName) } + return nil + } + year := t.Year() + if year < 70 { + // treats "0018" as 2018 + t = t.AddDate(2000, 0, 0) + } else if year < 100 { + // treats "0099" as 1999 + t = t.AddDate(1900, 0, 0) } - writer.WriteInt64Field(col.Name, t.Unix()/60/60/24) + writer.WriteInt64Field(colName, t.UTC().Unix()/60/60/24) return nil 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. - v, ok := col.Value.(string) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for datetime column %s", - col.Value, - col.Name) + value := datum.GetValue() + var v types.Time + switch val := value.(type) { + case string: + if val == "CURRENT_TIMESTAMP" { + writer.WriteInt64Field(colName, 0) + return nil + } else { + t, err := types.StrToDateTime(types.DefaultStmtNoWarningContext.WithLocation(c.config.TimeZone), val, ft.GetDecimal()) + if err != nil { + return errors.WrapError( + errors.ErrDebeziumEncodeFailed, + err) + } + v = t + } + case types.Time: + v = val + default: + return errors.Trace( + errors.ErrDebeziumEncodeFailed) } - - t, err := time.Parse("2006-01-02 15:04:05.999999", v) + gt, err := v.GoTime(time.UTC) if err != nil { - // For example, time may be 1000-00-00 if mysql.HasNotNullFlag(ft.GetFlag()) { - writer.WriteInt64Field(col.Name, 0) - return nil + writer.WriteInt64Field(colName, 0) } else { - writer.WriteNullField(col.Name) - return nil + writer.WriteNullField(colName) } + return nil + } + year := gt.Year() + if year < 70 { + // treats "0018" as 2018 + gt = gt.AddDate(2000, 0, 0) + } else if year < 100 { + // treats "0099" as 1999 + gt = gt.AddDate(1900, 0, 0) } - if ft.GetDecimal() <= 3 { - writer.WriteInt64Field(col.Name, t.UnixMilli()) - return nil + writer.WriteInt64Field(colName, gt.UnixMilli()) } else { - writer.WriteInt64Field(col.Name, t.UnixMicro()) - return nil + writer.WriteInt64Field(colName, gt.UnixMicro()) } + return nil case mysql.TypeTimestamp: // Debezium behavior from doc: @@ -441,80 +617,108 @@ func (c *dbzCodec) writeDebeziumFieldValue( // > 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. - v, ok := col.Value.(string) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for timestamp column %s", - col.Value, - col.Name) + value := datum.GetValue() + var v types.Time + switch val := value.(type) { + case string: + if val == "CURRENT_TIMESTAMP" { + if mysql.HasNotNullFlag(ft.GetFlag()) { + writer.WriteStringField(colName, "1970-01-01T00:00:00Z") + } else { + writer.WriteNullField(colName) + } + return nil + } else { + t, err := types.StrToDateTime(types.DefaultStmtNoWarningContext.WithLocation(c.config.TimeZone), val, ft.GetDecimal()) + if err != nil { + return errors.WrapError( + errors.ErrDebeziumEncodeFailed, + err) + } + v = t + } + case types.Time: + v = val } - - 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 - if mysql.HasNotNullFlag(ft.GetFlag()) { - t = time.Unix(0, 0) + if v.Compare(types.MinTimestamp) < 0 { + if row.IsNull(idx) { + writer.WriteNullField(colName) } else { - writer.WriteNullField(col.Name) - return nil + writer.WriteStringField(colName, "1970-01-01T00:00:00Z") } + return nil } - - str := t.UTC().Format("2006-01-02T15:04:05") + gt, err := v.GoTime(c.config.TimeZone) + if err != nil { + return errors.WrapError( + errors.ErrDebeziumEncodeFailed, + err) + } + str := gt.UTC().Format("2006-01-02T15:04:05") fsp := ft.GetDecimal() if fsp > 0 { - tmp := fmt.Sprintf(".%06d", t.Nanosecond()/1000) + tmp := fmt.Sprintf(".%06d", gt.Nanosecond()/1000) str = str + tmp[:1+fsp] } str += "Z" - - writer.WriteStringField(col.Name, str) + writer.WriteStringField(colName, str) return nil 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. - v, ok := col.Value.(string) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for time column %s", - col.Value, - col.Name) - } + d := datum.GetMysqlDuration() + writer.WriteInt64Field(colName, d.Microseconds()) + return nil - d, _, _, err := types.StrToDuration(types.DefaultStmtNoWarningContext, v, ft.GetDecimal()) - if err != nil { - return cerror.WrapError( - cerror.ErrDebeziumEncodeFailed, - err) + case mysql.TypeLonglong, mysql.TypeLong, mysql.TypeInt24, mysql.TypeShort, mysql.TypeTiny: + // Note: Although Debezium's doc claims to use INT32 for INT, but it + // actually uses INT64. Debezium also uses INT32 for SMALLINT. + maxValue := types.GetMaxValue(ft) + minValue := types.GetMinValue(ft) + isUnsigned := mysql.HasUnsignedFlag(colInfo.GetFlag()) + if isUnsigned { + v := datum.GetUint64() + if ft.GetType() == mysql.TypeLonglong && v == maxValue.GetUint64() || v > maxValue.GetUint64() { + writer.WriteAnyField(colName, -1) + } else { + writer.WriteInt64Field(colName, int64(v)) + } + } else { + v := datum.GetInt64() + if v < minValue.GetInt64() || v > maxValue.GetInt64() { + writer.WriteAnyField(colName, -1) + } else { + writer.WriteInt64Field(colName, v) + } } + return nil - writer.WriteInt64Field(col.Name, d.Microseconds()) + case mysql.TypeDouble: + v := datum.GetFloat64() + writer.WriteFloat64Field(colName, v) return nil - case mysql.TypeLonglong: - if col.Flag.IsUnsigned() { - // Handle with BIGINT UNSIGNED. - // Debezium always produce INT64 instead of UINT64 for BIGINT. - v, ok := col.Value.(uint64) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for unsigned bigint column %s", - col.Value, - col.Name) - } + case mysql.TypeFloat: + v := datum.GetFloat32() + writer.WriteFloat32Field(colName, v) + return nil + + case mysql.TypeYear: + v := datum.GetInt64() + writer.WriteInt64Field(colName, v) + return nil - writer.WriteInt64Field(col.Name, int64(v)) - return nil - } case mysql.TypeTiDBVectorFloat32: - v := col.Value.(types.VectorFloat32).String() - writer.WriteStringField(col.Name, v) + v := datum.GetVectorFloat32() + writer.WriteStringField(colName, v.String()) return nil } - - writer.WriteAnyField(col.Name, col.Value) + // NOTICE: GetValue() may return some types that go sql not support, which will cause sink DML fail + // Make specified convert upper if you need + // Go sql support type ref to: https://github.com/golang/go/blob/go1.17.4/src/database/sql/driver/types.go#L236 + writer.WriteAnyField(colName, fmt.Sprintf("%v", datum.GetValue())) return nil } @@ -523,8 +727,145 @@ func (c *dbzCodec) writeBinaryField(writer *util.JSONWriter, fieldName string, v writer.WriteBase64StringField(fieldName, value) } -func (c *dbzCodec) EncodeRowChangedEvent( - e *common.RowChangedEvent, +func (c *dbzCodec) writeSourceSchema(writer *util.JSONWriter) { + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "struct") + writer.WriteArrayField("fields", func() { + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "version") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "connector") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "name") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int64") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "ts_ms") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("name", "io.debezium.data.Enum") + writer.WriteIntField("version", 1) + writer.WriteObjectField("parameters", func() { + writer.WriteStringField("allowed", "true,last,false,incremental") + }) + writer.WriteStringField("default", "false") + writer.WriteStringField("field", "snapshot") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "db") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "sequence") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "table") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int64") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "server_id") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "gtid") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "file") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int64") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "pos") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int32") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "row") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int64") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "thread") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "query") + }) + }) + writer.WriteBoolField("optional", false) + writer.WriteStringField("name", "io.debezium.connector.mysql.Source") + writer.WriteStringField("field", "source") + }) +} + +// EncodeKey encode RowChangedEvent into key message +func (c *dbzCodec) EncodeKey( + e *commonEvent.RowEvent, + dest io.Writer, +) error { + // schema field describes the structure of the primary key, or the unique key if the table does not have a primary key, for the table that was changed. + // see https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-events + jWriter := util.BorrowJSONWriter(dest) + defer util.ReturnJSONWriter(jWriter) + + var err error + jWriter.WriteObject(func() { + jWriter.WriteObjectField("payload", func() { + columns := e.TableInfo.GetColumns() + row := e.GetRows() + if e.IsDelete() { + row = e.GetPreRows() + } + for idx, colInfo := range columns { + if colInfo != nil && e.TableInfo.IsHandleKey(colInfo.ID) { + err = c.writeDebeziumFieldValue(jWriter, row, idx, colInfo) + } + } + }) + if !c.config.DebeziumDisableSchema { + jWriter.WriteObjectField("schema", func() { + jWriter.WriteStringField("type", "struct") + jWriter.WriteStringField("name", + fmt.Sprintf("%s.Key", getSchemaTopicName(c.clusterID, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName()))) + jWriter.WriteBoolField("optional", false) + jWriter.WriteArrayField("fields", func() { + columns := e.TableInfo.GetColumns() + for _, colInfo := range columns { + if colInfo != nil && e.TableInfo.IsHandleKey(colInfo.ID) { + c.writeDebeziumFieldSchema(jWriter, colInfo) + } + } + }) + }) + } + }) + return err +} + +// EncodeValue encode RowChangedEvent into value message +func (c *dbzCodec) EncodeValue( + e *commonEvent.RowEvent, dest io.Writer, ) error { jWriter := util.BorrowJSONWriter(dest) @@ -564,7 +905,6 @@ func (c *dbzCodec) EncodeRowChangedEvent( // 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: @@ -584,18 +924,18 @@ func (c *dbzCodec) EncodeRowChangedEvent( // 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.writeDebeziumFieldValues(jWriter, "after", e.GetColumns(), e.TableInfo) + err = c.writeDebeziumFieldValues(jWriter, "after", e.GetRows(), e.TableInfo) } else if e.IsDelete() { jWriter.WriteStringField("op", "d") jWriter.WriteNullField("after") - err = c.writeDebeziumFieldValues(jWriter, "before", e.GetPreColumns(), e.TableInfo) + err = c.writeDebeziumFieldValues(jWriter, "before", e.GetPreRows(), e.TableInfo) } else if e.IsUpdate() { jWriter.WriteStringField("op", "u") if c.config.DebeziumOutputOldValue { - err = c.writeDebeziumFieldValues(jWriter, "before", e.GetPreColumns(), e.TableInfo) + err = c.writeDebeziumFieldValues(jWriter, "before", e.GetPreRows(), e.TableInfo) } if err == nil { - err = c.writeDebeziumFieldValues(jWriter, "after", e.GetColumns(), e.TableInfo) + err = c.writeDebeziumFieldValues(jWriter, "after", e.GetRows(), e.TableInfo) } } }) @@ -604,10 +944,8 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteObjectField("schema", func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Envelope", - c.clusterID, - e.TableInfo.GetSchemaName(), - e.TableInfo.GetTableName())) + jWriter.WriteStringField("name", + fmt.Sprintf("%s.Envelope", getSchemaTopicName(c.clusterID, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName()))) jWriter.WriteIntField("version", 1) jWriter.WriteArrayField("fields", func() { // schema is the same for `before` and `after`. So we build a new buffer to @@ -616,17 +954,16 @@ func (c *dbzCodec) EncodeRowChangedEvent( { fieldsBuf := &bytes.Buffer{} fieldsWriter := util.BorrowJSONWriter(fieldsBuf) - var validCols []*common.Column - if e.IsInsert() { - validCols = e.GetColumns() - } else if e.IsDelete() { - validCols = e.GetPreColumns() - } else if e.IsUpdate() { - validCols = e.GetColumns() + for _, col := range e.TableInfo.GetColumns() { + c.writeDebeziumFieldSchema(fieldsWriter, col) } - colInfos := e.TableInfo.GetColInfosForRowChangedEvent() - for i, col := range validCols { - c.writeDebeziumFieldSchema(fieldsWriter, col, colInfos[i].Ft) + if e.TableInfo.HasVirtualColumns() { + for _, colInfo := range e.TableInfo.GetColumns() { + if commonType.IsColCDCVisible(colInfo) { + continue + } + c.writeDebeziumFieldSchema(fieldsWriter, colInfo) + } } util.ReturnJSONWriter(fieldsWriter) fieldsJSON = fieldsBuf.String() @@ -634,10 +971,8 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Value", - c.clusterID, - e.TableInfo.GetSchemaName(), - e.TableInfo.GetTableName())) + jWriter.WriteStringField("name", + fmt.Sprintf("%s.Value", getSchemaTopicName(c.clusterID, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName()))) jWriter.WriteStringField("field", "before") jWriter.WriteArrayField("fields", func() { jWriter.WriteRaw(fieldsJSON) @@ -646,115 +981,545 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Value", - c.clusterID, - e.TableInfo.GetSchemaName(), - e.TableInfo.GetTableName())) + jWriter.WriteStringField("name", + fmt.Sprintf("%s.Value", getSchemaTopicName(c.clusterID, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName()))) jWriter.WriteStringField("field", "after") jWriter.WriteArrayField("fields", func() { jWriter.WriteRaw(fieldsJSON) }) }) + c.writeSourceSchema(jWriter) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("type", "string") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("field", "op") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("type", "int64") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("field", "ts_ms") + }) jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "struct") jWriter.WriteArrayField("fields", func() { jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "string") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "version") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "connector") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "name") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "int64") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "ts_ms") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("name", "io.debezium.data.Enum") - jWriter.WriteIntField("version", 1) - jWriter.WriteObjectField("parameters", func() { - jWriter.WriteStringField("allowed", "true,last,false,incremental") - }) - jWriter.WriteStringField("default", "false") - jWriter.WriteStringField("field", "snapshot") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "db") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "sequence") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "table") + jWriter.WriteStringField("field", "id") }) jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "int64") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "server_id") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "gtid") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "file") + jWriter.WriteStringField("field", "total_order") }) jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "int64") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "pos") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "int32") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "row") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "int64") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "thread") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "query") + jWriter.WriteStringField("field", "data_collection_order") }) - // Below are extra TiDB fields - // jWriter.WriteObjectElement(func() { - // jWriter.WriteStringField("type", "int64") - // jWriter.WriteBoolField("optional", false) - // jWriter.WriteStringField("field", "commit_ts") - // }) - // jWriter.WriteObjectElement(func() { - // jWriter.WriteStringField("type", "string") - // jWriter.WriteBoolField("optional", false) - // jWriter.WriteStringField("field", "cluster_id") - // }) }) + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("name", "event.block") + jWriter.WriteIntField("version", 1) + jWriter.WriteStringField("field", "transaction") + }) + }) + }) + } + }) + return err +} + +// EncodeDDLEvent encode DDLEvent into debezium change event +func (c *dbzCodec) EncodeDDLEvent( + e *commonEvent.DDLEvent, + keyDest io.Writer, + dest io.Writer, +) error { + keyJWriter := util.BorrowJSONWriter(keyDest) + jWriter := util.BorrowJSONWriter(dest) + defer util.ReturnJSONWriter(keyJWriter) + defer util.ReturnJSONWriter(jWriter) + + commitTime := oracle.GetTimeFromTS(e.FinishedTs) + var changeType string + // refer to: https://docs.pingcap.com/tidb/dev/mysql-compatibility#ddl-operations + switch e.GetDDLType() { + case timodel.ActionCreateSchema, + timodel.ActionCreateTable, + timodel.ActionCreateTables, + timodel.ActionCreateView: + changeType = "CREATE" + case timodel.ActionAddColumn, + timodel.ActionModifyColumn, + timodel.ActionDropColumn, + timodel.ActionMultiSchemaChange, + timodel.ActionAddTablePartition, + timodel.ActionRemovePartitioning, + timodel.ActionReorganizePartition, + timodel.ActionExchangeTablePartition, + timodel.ActionAlterTablePartitioning, + timodel.ActionTruncateTablePartition, + timodel.ActionDropTablePartition, + timodel.ActionRebaseAutoID, + timodel.ActionSetDefaultValue, + timodel.ActionModifyTableComment, + timodel.ActionModifyTableCharsetAndCollate, + timodel.ActionModifySchemaCharsetAndCollate, + timodel.ActionAddIndex, + timodel.ActionAlterIndexVisibility, + timodel.ActionRenameIndex, + timodel.ActionRenameTable, + timodel.ActionRenameTables, + timodel.ActionAddPrimaryKey, + timodel.ActionDropPrimaryKey: + changeType = "ALTER" + case timodel.ActionDropSchema, + timodel.ActionDropTable, + timodel.ActionDropIndex, + timodel.ActionDropView, + timodel.ActionTruncateTable: + changeType = "DROP" + default: + return errors.ErrDDLUnsupportType.GenWithStackByArgs(e.Type, e.Query) + } + + var err error + dbName, tableName := getDBTableName(e) + // message key + keyJWriter.WriteObject(func() { + keyJWriter.WriteObjectField("payload", func() { + if e.GetDDLType() == timodel.ActionDropTable { + keyJWriter.WriteStringField("databaseName", e.ExtraSchemaName) + } else { + keyJWriter.WriteStringField("databaseName", dbName) + } + }) + if !c.config.DebeziumDisableSchema { + keyJWriter.WriteObjectField("schema", func() { + keyJWriter.WriteStringField("type", "struct") + keyJWriter.WriteStringField("name", "io.debezium.connector.mysql.SchemaChangeKey") + keyJWriter.WriteBoolField("optional", false) + keyJWriter.WriteIntField("version", 1) + keyJWriter.WriteArrayField("fields", func() { + keyJWriter.WriteObjectElement(func() { + keyJWriter.WriteStringField("field", "databaseName") + keyJWriter.WriteBoolField("optional", false) + keyJWriter.WriteStringField("type", "string") + }) + }) + }) + } + }) + + // message value + 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.WriteStringField("snapshot", "false") + if e.TableInfo == nil { + jWriter.WriteStringField("db", "") + jWriter.WriteStringField("table", "") + } else { + jWriter.WriteStringField("db", dbName) + jWriter.WriteStringField("table", tableName) + } + 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") + + // The followings are TiDB extended fields + jWriter.WriteUint64Field("commit_ts", e.GetCommitTs()) + jWriter.WriteStringField("cluster_id", c.clusterID) + }) + jWriter.WriteInt64Field("ts_ms", c.nowFunc().UnixMilli()) + + if e.GetDDLType() == timodel.ActionDropTable { + jWriter.WriteStringField("databaseName", e.ExtraSchemaName) + } else { + jWriter.WriteStringField("databaseName", dbName) + } + jWriter.WriteNullField("schemaName") + jWriter.WriteStringField("ddl", e.Query) + jWriter.WriteArrayField("tableChanges", func() { + // return early if there is no table changes + if tableName == "" { + return + } + jWriter.WriteObjectElement(func() { + // Describes the kind of change. The value is one of the following: + // CREATE: Table created. + // ALTER: Table modified. + // DROP: Table deleted. + jWriter.WriteStringField("type", changeType) + // In the case of a table rename, this identifier is a concatenation of , table names. + if e.GetDDLType() == timodel.ActionRenameTable { + jWriter.WriteStringField("id", fmt.Sprintf("\"%s\".\"%s\",\"%s\".\"%s\"", + e.ExtraSchemaName, + e.ExtraTableName, + dbName, + tableName)) + } else { + jWriter.WriteStringField("id", fmt.Sprintf("\"%s\".\"%s\"", + dbName, + tableName)) + } + // return early if there is no table info + if e.GetDDLType() == timodel.ActionDropTable { + jWriter.WriteNullField("table") + return + } + jWriter.WriteObjectField("table", func() { + jWriter.WriteStringField("defaultCharsetName", e.TableInfo.Charset) + jWriter.WriteArrayField("primaryKeyColumnNames", func() { + for _, pk := range e.TableInfo.GetPrimaryKeyColumnNames() { + jWriter.WriteStringElement(pk) + } + }) + jWriter.WriteArrayField("columns", func() { + parseColumns(e.Query, e.TableInfo.GetColumns()) + for pos, col := range e.TableInfo.GetColumns() { + if col.Hidden { + continue + } + jWriter.WriteObjectElement(func() { + flag := col.GetFlag() + jdbcType := common.MySQLType2JdbcType(col.GetType(), mysql.HasBinaryFlag(flag)) + expression, name := getExpressionAndName(col.FieldType) + jWriter.WriteStringField("name", col.Name.O) + jWriter.WriteIntField("jdbcType", int(jdbcType)) + jWriter.WriteNullField("nativeType") + if col.Comment != "" { + jWriter.WriteStringField("comment", col.Comment) + } else { + jWriter.WriteNullField("comment") + } + if col.DefaultValue == nil { + jWriter.WriteNullField("defaultValueExpression") + } else { + v, ok := col.DefaultValue.(string) + if ok { + if strings.ToUpper(v) == "CURRENT_TIMESTAMP" { + // https://debezium.io/documentation/reference/3.0/connectors/mysql.html#mysql-temporal-types + jWriter.WriteAnyField("defaultValueExpression", "1970-01-01 00:00:00") + } else if v == "" { + jWriter.WriteNullField("defaultValueExpression") + } else if col.DefaultValueBit != nil { + jWriter.WriteStringField("defaultValueExpression", parseBit(v, col.GetFlen())) + } else { + jWriter.WriteStringField("defaultValueExpression", v) + } + } else { + jWriter.WriteAnyField("defaultValueExpression", col.DefaultValue) + } + } + elems := col.GetElems() + if len(elems) != 0 { + // Format is ENUM ('e1', 'e2') or SET ('e1', 'e2') + jWriter.WriteArrayField("enumValues", func() { + for _, ele := range elems { + jWriter.WriteStringElement(fmt.Sprintf("'%s'", ele)) + } + }) + } else { + jWriter.WriteNullField("enumValues") + } + + jWriter.WriteStringField("typeName", name) + jWriter.WriteStringField("typeExpression", expression) + + charsetName := getCharset(col.FieldType) + if charsetName != "" { + jWriter.WriteStringField("charsetName", charsetName) + } else { + jWriter.WriteNullField("charsetName") + } + + length := getLen(col.FieldType) + if length != -1 { + jWriter.WriteIntField("length", length) + } else { + jWriter.WriteNullField("length") + } + + scale := getScale(col.FieldType) + if scale != -1 { + jWriter.WriteFloat64Field("scale", scale) + } else { + jWriter.WriteNullField("scale") + } + jWriter.WriteIntField("position", pos+1) + jWriter.WriteBoolField("optional", !mysql.HasNotNullFlag(flag)) + + updateNowWithTimestamp := mysql.HasOnUpdateNowFlag(flag) && jdbcType == common.JavaSQLTypeTIMESTAMP_WITH_TIMEZONE + autoIncrementFlag := mysql.HasAutoIncrementFlag(flag) || updateNowWithTimestamp + + jWriter.WriteBoolField("autoIncremented", autoIncrementFlag) + jWriter.WriteBoolField("generated", autoIncrementFlag) + }) + } + }) + jWriter.WriteNullField("comment") + }) + }) + }) + }) + + if !c.config.DebeziumDisableSchema { + jWriter.WriteObjectField("schema", func() { + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "struct") + jWriter.WriteIntField("version", 1) + jWriter.WriteStringField("name", "io.debezium.connector.mysql.SchemaChangeValue") + jWriter.WriteArrayField("fields", func() { + c.writeSourceSchema(jWriter) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "ts_ms") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "int64") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "databaseName") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "schemaName") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "ddl") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "tableChanges") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("name", "io.debezium.connector.mysql.Source") - jWriter.WriteStringField("field", "source") + jWriter.WriteStringField("type", "array") + jWriter.WriteObjectField("items", func() { + jWriter.WriteStringField("name", "io.debezium.connector.schema.Change") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "struct") + jWriter.WriteIntField("version", 1) + jWriter.WriteArrayField("fields", func() { + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "type") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "id") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "table") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "struct") + jWriter.WriteStringField("name", "io.debezium.connector.schema.Table") + jWriter.WriteIntField("version", 1) + jWriter.WriteArrayField("fields", func() { + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "defaultCharsetName") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "primaryKeyColumnNames") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "array") + jWriter.WriteObjectField("items", func() { + jWriter.WriteStringField("type", "string") + jWriter.WriteBoolField("optional", false) + }) + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "columns") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "array") + jWriter.WriteObjectField("items", func() { + jWriter.WriteStringField("name", "io.debezium.connector.schema.Column") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "struct") + jWriter.WriteIntField("version", 1) + jWriter.WriteArrayField("fields", func() { + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "name") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "jdbcType") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "nativeType") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "typeName") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "typeExpression") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "charsetName") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "length") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "scale") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "position") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "optional") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "boolean") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "autoIncremented") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "boolean") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "generated") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "boolean") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "comment") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "defaultValueExpression") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "enumValues") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "array") + jWriter.WriteObjectField("items", func() { + jWriter.WriteStringField("type", "string") + jWriter.WriteBoolField("optional", false) + }) + }) + }) + }) + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "comment") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + }) + }) + }) + }) }) + }) + }) + } + }) + return err +} + +// EncodeCheckpointEvent encode checkpointTs into debezium change event +func (c *dbzCodec) EncodeCheckpointEvent( + ts uint64, + keyDest io.Writer, + dest io.Writer, +) error { + keyJWriter := util.BorrowJSONWriter(keyDest) + jWriter := util.BorrowJSONWriter(dest) + defer util.ReturnJSONWriter(keyJWriter) + defer util.ReturnJSONWriter(jWriter) + commitTime := oracle.GetTimeFromTS(ts) + var err error + // message key + keyJWriter.WriteObject(func() { + keyJWriter.WriteObjectField("payload", func() {}) + if !c.config.DebeziumDisableSchema { + keyJWriter.WriteObjectField("schema", func() { + keyJWriter.WriteStringField("type", "struct") + keyJWriter.WriteStringField("name", + fmt.Sprintf("%s.%s.Key", common.SanitizeName(c.clusterID), "watermark")) + keyJWriter.WriteBoolField("optional", false) + keyJWriter.WriteArrayField("fields", func() { + }) + }) + } + }) + // message value + 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) + // 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()) + // snapshot field is a string of true,last,false,incremental + jWriter.WriteStringField("snapshot", "false") + jWriter.WriteStringField("db", "") + jWriter.WriteStringField("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") + + // The followings are TiDB extended fields + jWriter.WriteUint64Field("commit_ts", ts) + 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") + jWriter.WriteStringField("op", "m") + }) + + if !c.config.DebeziumDisableSchema { + jWriter.WriteObjectField("schema", func() { + jWriter.WriteStringField("type", "struct") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("name", + fmt.Sprintf("%s.%s.Envelope", common.SanitizeName(c.clusterID), "watermark")) + jWriter.WriteIntField("version", 1) + jWriter.WriteArrayField("fields", func() { + c.writeSourceSchema(jWriter) jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "string") jWriter.WriteBoolField("optional", false) @@ -793,6 +1558,5 @@ func (c *dbzCodec) EncodeRowChangedEvent( }) } }) - return err } diff --git a/pkg/sink/codec/debezium/codec_test.go b/pkg/sink/codec/debezium/codec_test.go new file mode 100644 index 000000000..ef9308b1d --- /dev/null +++ b/pkg/sink/codec/debezium/codec_test.go @@ -0,0 +1,1502 @@ +// 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 ( + "bytes" + "testing" + "time" + + "github.com/pingcap/ticdc/pkg/common/columnselector" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + timodel "github.com/pingcap/tidb/pkg/meta/model" + "github.com/stretchr/testify/require" + "github.com/thanhpk/randstr" +) + +func TestDDLEvent(t *testing.T) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + + helper := commonEvent.NewEventTestHelper(t) + defer helper.Close() + + helper.Tk().MustExec("use test") + + helper.DDL2Job(`create table test.table1(id int(10) primary key)`) + job := helper.DDL2Job(`RENAME TABLE test.table1 to test.table2`) + tableInfo := helper.GetTableInfo(job) + + e := &commonEvent.DDLEvent{ + FinishedTs: 1, + TableInfo: tableInfo, + SchemaName: "test", + TableName: "table2", + ExtraSchemaName: "test", + ExtraTableName: "table1", + Type: byte(timodel.ActionNone), + Query: job.Query, + } + keyBuf := bytes.NewBuffer(nil) + buf := bytes.NewBuffer(nil) + err := codec.EncodeDDLEvent(e, keyBuf, buf) + require.ErrorIs(t, err, errors.ErrDDLUnsupportType) + + e = &commonEvent.DDLEvent{ + FinishedTs: 1, + TableInfo: tableInfo, + SchemaName: "test", + TableName: "table2", + ExtraSchemaName: "test", + ExtraTableName: "table1", + Query: job.Query, + Type: byte(timodel.ActionRenameTable), + } + keyBuf.Reset() + buf.Reset() + codec.config.DebeziumDisableSchema = false + err = codec.EncodeDDLEvent(e, keyBuf, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "databaseName": "test" + }, + "schema": { + "type": "struct", + "name": "io.debezium.connector.mysql.SchemaChangeKey", + "optional": false, + "version": 1, + "fields": [ + { + "field": "databaseName", + "optional": false, + "type": "string" + } + ] + } + }`, keyBuf.String()) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "test", + "table": "table2", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "databaseName": "test", + "schemaName": null, + "ddl": "RENAME TABLE test.table1 to test.table2", + "tableChanges": [ + { + "type": "ALTER", + "id": "\"test\".\"table1\",\"test\".\"table2\"", + "table": { + "defaultCharsetName": "utf8mb4", + "primaryKeyColumnNames": ["id"], + "columns": [ + { + "name": "id", + "jdbcType": 4, + "nativeType": null, + "comment": null, + "defaultValueExpression": null, + "enumValues": null, + "typeName": "INT", + "typeExpression": "INT", + "charsetName": null, + "length": 10, + "scale": null, + "position": 1, + "optional": false, + "autoIncremented": false, + "generated": false + } + ], + "comment": null + } + } + ] + }, + "schema": { + "optional": false, + "type": "struct", + "version": 1, + "name": "io.debezium.connector.mysql.SchemaChangeValue", + "fields": [ + { + "field": "source", + "name": "io.debezium.connector.mysql.Source", + "optional": false, + "type": "struct", + "fields": [ + { + "field": "version", + "optional": false, + "type": "string" + }, + { + "field": "connector", + "optional": false, + "type": "string" + }, + { + "field": "name", + "optional": false, + "type": "string" + }, + { + "field": "ts_ms", + "optional": false, + "type": "int64" + }, + { + "field": "snapshot", + "optional": true, + "type": "string", + "parameters": { + "allowed": "true,last,false,incremental" + }, + "default": "false", + "name": "io.debezium.data.Enum", + "version": 1 + }, + { + "field": "db", + "optional": false, + "type": "string" + }, + { + "field": "sequence", + "optional": true, + "type": "string" + }, + { + "field": "table", + "optional": true, + "type": "string" + }, + { + "field": "server_id", + "optional": false, + "type": "int64" + }, + { + "field": "gtid", + "optional": true, + "type": "string" + }, + { + "field": "file", + "optional": false, + "type": "string" + }, + { + "field": "pos", + "optional": false, + "type": "int64" + }, + { + "field": "row", + "optional": false, + "type": "int32" + }, + { + "field": "thread", + "optional": true, + "type": "int64" + }, + { + "field": "query", + "optional": true, + "type": "string" + } + ] + }, + { + "field": "ts_ms", + "optional": false, + "type": "int64" + }, + { + "field": "databaseName", + "optional": true, + "type": "string" + }, + { + "field": "schemaName", + "optional": true, + "type": "string" + }, + { + "field": "ddl", + "optional": true, + "type": "string" + }, + { + "field": "tableChanges", + "optional": false, + "type": "array", + "items": { + "name": "io.debezium.connector.schema.Change", + "optional": false, + "type": "struct", + "version": 1, + "fields": [ + { + "field": "type", + "optional": false, + "type": "string" + }, + { + "field": "id", + "optional": false, + "type": "string" + }, + { + "field": "table", + "optional": true, + "type": "struct", + "name": "io.debezium.connector.schema.Table", + "version": 1, + "fields": [ + { + "field": "defaultCharsetName", + "optional": true, + "type": "string" + }, + { + "field": "primaryKeyColumnNames", + "optional": true, + "type": "array", + "items": { + "type": "string", + "optional": false + } + }, + { + "field": "columns", + "optional": false, + "type": "array", + "items": { + "name": "io.debezium.connector.schema.Column", + "optional": false, + "type": "struct", + "version": 1, + "fields": [ + { + "field": "name", + "optional": false, + "type": "string" + }, + { + "field": "jdbcType", + "optional": false, + "type": "int32" + }, + { + "field": "nativeType", + "optional": true, + "type": "int32" + }, + { + "field": "typeName", + "optional": false, + "type": "string" + }, + { + "field": "typeExpression", + "optional": true, + "type": "string" + }, + { + "field": "charsetName", + "optional": true, + "type": "string" + }, + { + "field": "length", + "optional": true, + "type": "int32" + }, + { + "field": "scale", + "optional": true, + "type": "int32" + }, + { + "field": "position", + "optional": false, + "type": "int32" + }, + { + "field": "optional", + "optional": true, + "type": "boolean" + }, + { + "field": "autoIncremented", + "optional": true, + "type": "boolean" + }, + { + "field": "generated", + "optional": true, + "type": "boolean" + }, + { + "field": "comment", + "optional": true, + "type": "string" + }, + { + "field": "defaultValueExpression", + "optional": true, + "type": "string" + }, + { + "field": "enumValues", + "optional": true, + "type": "array", + "items": { + "type": "string", + "optional": false + } + } + ] + } + }, + { + "field": "comment", + "optional": true, + "type": "string" + } + ] + } + ] + } + } + ] + } + }`, buf.String()) + + codec.config.DebeziumDisableSchema = true + + job = helper.DDL2Job("CREATE TABLE test.table1(id int(10) primary key)") + tableInfo = helper.GetTableInfo(job) + e = &commonEvent.DDLEvent{ + FinishedTs: 1, + TableInfo: tableInfo, + SchemaName: job.SchemaName, + TableName: job.TableName, + Query: job.Query, + Type: byte(timodel.ActionCreateTable), + } + keyBuf.Reset() + buf.Reset() + err = codec.EncodeDDLEvent(e, keyBuf, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "databaseName": "test" + } + }`, keyBuf.String()) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "test", + "table": "table1", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "databaseName": "test", + "schemaName": null, + "ddl": "CREATE TABLE test.table1(id int(10) primary key)", + "tableChanges": [ + { + "type": "CREATE", + "id": "\"test\".\"table1\"", + "table": { + "defaultCharsetName": "utf8mb4", + "primaryKeyColumnNames": ["id"], + "columns": [ + { + "name": "id", + "jdbcType": 4, + "nativeType": null, + "comment": null, + "defaultValueExpression": null, + "enumValues": null, + "typeName": "INT", + "typeExpression": "INT", + "charsetName": null, + "length": 10, + "scale": null, + "position": 1, + "optional": false, + "autoIncremented": false, + "generated": false + } + ], + "comment": null + } + } + ] + } + }`, buf.String()) + + job = helper.DDL2Job("DROP TABLE test.table2") + tableInfo = helper.GetTableInfo(job) + e = &commonEvent.DDLEvent{ + FinishedTs: 1, + TableInfo: tableInfo, + ExtraSchemaName: job.SchemaName, + ExtraTableName: job.TableName, + Query: job.Query, + Type: byte(timodel.ActionDropTable), + } + keyBuf.Reset() + buf.Reset() + err = codec.EncodeDDLEvent(e, keyBuf, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "databaseName": "test" + } + }`, keyBuf.String()) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "", + "table": "", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "databaseName": "test", + "schemaName": null, + "ddl": "DROP TABLE test.table2", + "tableChanges": [] + } + }`, buf.String()) +} + +func TestCheckPointEvent(t *testing.T) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.DebeziumDisableSchema = false + + var ts uint64 = 3 + keyBuf := bytes.NewBuffer(nil) + buf := bytes.NewBuffer(nil) + err := codec.EncodeCheckpointEvent(ts, keyBuf, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": {}, + "schema": { + "fields": [], + "optional": false, + "name": "test_cluster.watermark.Key", + "type": "struct" + } + }`, keyBuf.String()) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "", + "table": "", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 3, + "cluster_id": "test_cluster" + }, + "op":"m", + "ts_ms": 1701326309000, + "transaction": null + }, + "schema": { + "type": "struct", + "optional": false, + "name": "test_cluster.watermark.Envelope", + "version": 1, + "fields": [ + { + "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" + } + ] + } + }`, buf.String()) +} + +func TestEncodeInsert(t *testing.T) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.DebeziumDisableSchema = true + codec.config.DebeziumOutputOldValue = false + + helper := commonEvent.NewEventTestHelper(t) + defer helper.Close() + + helper.Tk().MustExec("use test") + + job := helper.DDL2Job(`create table test.table1(tiny tinyint primary key)`) + dmlEvent := helper.DML2Event("test", "table1", `insert into test.table1 values (1)`) + require.NotNil(t, dmlEvent) + row, ok := dmlEvent.GetNextRow() + require.True(t, ok) + tableInfo := helper.GetTableInfo(job) + + e := &commonEvent.RowEvent{ + TableInfo: tableInfo, + CommitTs: 1, + Event: row, + ColumnSelector: columnselector.NewDefaultColumnSelector(), + Callback: func() {}, + } + + buf := bytes.NewBuffer(nil) + keyBuf := bytes.NewBuffer(nil) + err := codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + } + } + `, keyBuf.String()) + err = codec.EncodeValue(e, buf) + 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 + } + } + `, buf.String()) + + codec.config.DebeziumDisableSchema = false + keyBuf.Reset() + err = codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + }, + "schema": { + "fields": [ + { + "field":"tiny", + "optional":false, + "type":"int16" + } + ], + "name": "test_cluster.test.table1.Key", + "optional": false, + "type":"struct" + } + } + `, keyBuf.String()) + buf.Reset() + err = codec.EncodeValue(e, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "test", + "table": "table1", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "transaction": null, + "op": "c", + "before": null, + "after": { "tiny": 1 } + }, + "schema": { + "type": "struct", + "optional": false, + "name": "test_cluster.test.table1.Envelope", + "version": 1, + "fields": [ + { + "type": "struct", + "optional": true, + "name": "test_cluster.test.table1.Value", + "field": "before", + "fields": [{ "type": "int16", "optional": false, "field": "tiny" }] + }, + { + "type": "struct", + "optional": true, + "name": "test_cluster.test.table1.Value", + "field": "after", + "fields": [{ "type": "int16", "optional": false, "field": "tiny" }] + }, + { + "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" + } + ] + } + } + `, buf.String()) +} + +func TestEncodeUpdate(t *testing.T) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.DebeziumDisableSchema = true + + helper := commonEvent.NewEventTestHelper(t) + defer helper.Close() + + helper.Tk().MustExec("use test") + + job := helper.DDL2Job(`create table test.table1(tiny tinyint primary key)`) + dmlEvent := helper.DML2Event("test", "table1", `insert into test.table1 values (2)`) + require.NotNil(t, dmlEvent) + insertRow, ok := dmlEvent.GetNextRow() + require.True(t, ok) + dmlEvent = helper.DML2Event("test", "table1", `update test.table1 set tiny=1 where tiny=2`) + require.NotNil(t, dmlEvent) + row, ok := dmlEvent.GetNextRow() + row.PreRow = insertRow.Row + require.True(t, ok) + tableInfo := helper.GetTableInfo(job) + + e := &commonEvent.RowEvent{ + TableInfo: tableInfo, + CommitTs: 1, + Event: row, + ColumnSelector: columnselector.NewDefaultColumnSelector(), + Callback: func() {}, + } + + buf := bytes.NewBuffer(nil) + keyBuf := bytes.NewBuffer(nil) + err := codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + } + } + `, keyBuf.String()) + + err = codec.EncodeValue(e, buf) + 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 + } + } + `, buf.String()) + + codec.config.DebeziumDisableSchema = false + keyBuf.Reset() + err = codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + }, + "schema": { + "fields": [ + { + "field":"tiny", + "optional":false, + "type":"int16" + } + ], + "name": "test_cluster.test.table1.Key", + "optional": false, + "type":"struct" + } + } + `, keyBuf.String()) + + buf.Reset() + err = codec.EncodeValue(e, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "test", + "table": "table1", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "transaction": null, + "op": "u", + "before": { "tiny": 2 }, + "after": { "tiny": 1 } + }, + "schema": { + "type": "struct", + "optional": false, + "name": "test_cluster.test.table1.Envelope", + "version": 1, + "fields": [ + { + "type": "struct", + "optional": true, + "name": "test_cluster.test.table1.Value", + "field": "before", + "fields": [{ "type": "int16", "optional": false, "field": "tiny" }] + }, + { + "type": "struct", + "optional": true, + "name": "test_cluster.test.table1.Value", + "field": "after", + "fields": [{ "type": "int16", "optional": false, "field": "tiny" }] + }, + { + "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" + } + ] + } + } + `, buf.String()) + + codec.config.DebeziumOutputOldValue = false + codec.config.DebeziumDisableSchema = true + + keyBuf.Reset() + err = codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + } + } + `, keyBuf.String()) + + buf.Reset() + err = codec.EncodeValue(e, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "test", + "table": "table1", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "transaction": null, + "op": "u", + "after": { "tiny": 1 } + } + } + `, buf.String()) +} + +func TestEncodeDelete(t *testing.T) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.DebeziumOutputOldValue = false + codec.config.DebeziumDisableSchema = true + + helper := commonEvent.NewEventTestHelper(t) + defer helper.Close() + + helper.Tk().MustExec("use test") + + job := helper.DDL2Job(`create table test.table1(tiny tinyint primary key)`) + dmlEvent := helper.DML2Event("test", "table1", `insert into test.table1 values (2)`) + require.NotNil(t, dmlEvent) + row, ok := dmlEvent.GetNextRow() + require.True(t, ok) + tableInfo := helper.GetTableInfo(job) + tmpRow := row.Row + row.Row = row.PreRow + row.PreRow = tmpRow + + e := &commonEvent.RowEvent{ + TableInfo: tableInfo, + CommitTs: 1, + Event: row, + ColumnSelector: columnselector.NewDefaultColumnSelector(), + Callback: func() {}, + } + buf := bytes.NewBuffer(nil) + keyBuf := bytes.NewBuffer(nil) + err := codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 2 + } + } + `, keyBuf.String()) + + err = codec.EncodeValue(e, buf) + 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 + } + } + `, buf.String()) + + codec.config.DebeziumDisableSchema = false + + keyBuf.Reset() + err = codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 2 + }, + "schema": { + "fields": [ + { + "field":"tiny", + "optional":false, + "type":"int16" + } + ], + "name": "test_cluster.test.table1.Key", + "optional": false, + "type":"struct" + } + } + `, keyBuf.String()) + + buf.Reset() + err = codec.EncodeValue(e, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "test", + "table": "table1", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "transaction": null, + "op": "d", + "after": null, + "before": { "tiny": 2 } + }, + "schema": { + "type": "struct", + "optional": false, + "name": "test_cluster.test.table1.Envelope", + "version": 1, + "fields": [ + { + "type": "struct", + "optional": true, + "name": "test_cluster.test.table1.Value", + "field": "before", + "fields": [{ "type": "int16", "optional": false, "field": "tiny" }] + }, + { + "type": "struct", + "optional": true, + "name": "test_cluster.test.table1.Value", + "field": "after", + "fields": [{ "type": "int16", "optional": false, "field": "tiny" }] + }, + { + "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" + } + ] + } + } + `, buf.String()) +} + +func BenchmarkEncodeOneTinyColumn(b *testing.B) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.DebeziumDisableSchema = true + + helper := commonEvent.NewEventTestHelper(b) + defer helper.Close() + + helper.Tk().MustExec("use test") + + job := helper.DDL2Job(`create table test.table1(tiny tinyint primary key)`) + dmlEvent := helper.DML2Event("test", "table1", `insert into test.table1 values (10)`) + row, _ := dmlEvent.GetNextRow() + tableInfo := helper.GetTableInfo(job) + + e := &commonEvent.RowEvent{ + TableInfo: tableInfo, + CommitTs: 1, + Event: row, + ColumnSelector: columnselector.NewDefaultColumnSelector(), + Callback: func() {}, + } + + keyBuf := bytes.NewBuffer(nil) + buf := bytes.NewBuffer(nil) + + b.ResetTimer() + for n := 0; n < b.N; n++ { + keyBuf.Reset() + buf.Reset() + codec.EncodeKey(e, keyBuf) + codec.EncodeValue(e, buf) + } +} + +func BenchmarkEncodeLargeText(b *testing.B) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.DebeziumDisableSchema = true + + helper := commonEvent.NewEventTestHelper(b) + defer helper.Close() + + helper.Tk().MustExec("use test") + + job := helper.DDL2Job(`create table test.table1(str varchar(1024))`) + dmlEvent := helper.DML2Event("test", "table1", `insert into test.table1 values ("`+randstr.String(1024)+`")`) + row, _ := dmlEvent.GetNextRow() + tableInfo := helper.GetTableInfo(job) + + e := &commonEvent.RowEvent{ + TableInfo: tableInfo, + CommitTs: 1, + Event: row, + ColumnSelector: columnselector.NewDefaultColumnSelector(), + Callback: func() {}, + } + keyBuf := bytes.NewBuffer(nil) + buf := bytes.NewBuffer(nil) + + b.ResetTimer() + for n := 0; n < b.N; n++ { + keyBuf.Reset() + buf.Reset() + codec.EncodeKey(e, keyBuf) + codec.EncodeValue(e, buf) + } +} + +func BenchmarkEncodeLargeBinary(b *testing.B) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.DebeziumDisableSchema = true + + helper := commonEvent.NewEventTestHelper(b) + defer helper.Close() + + helper.Tk().MustExec("use test") + + job := helper.DDL2Job(`create table test.table1(tiny varbinary(1024))`) + dmlEvent := helper.DML2Event("test", "table1", `insert into test.table1 values ("`+randstr.String(1024)+`")`) + row, _ := dmlEvent.GetNextRow() + tableInfo := helper.GetTableInfo(job) + + e := &commonEvent.RowEvent{ + TableInfo: tableInfo, + CommitTs: 1, + Event: row, + ColumnSelector: columnselector.NewDefaultColumnSelector(), + Callback: func() {}, + } + + keyBuf := bytes.NewBuffer(nil) + buf := bytes.NewBuffer(nil) + + b.ResetTimer() + for n := 0; n < b.N; n++ { + keyBuf.Reset() + buf.Reset() + codec.EncodeKey(e, keyBuf) + codec.EncodeValue(e, buf) + } +} diff --git a/pkg/sink/codec/debezium/debezium_test.go b/pkg/sink/codec/debezium/debezium_test.go new file mode 100644 index 000000000..59fc557ef --- /dev/null +++ b/pkg/sink/codec/debezium/debezium_test.go @@ -0,0 +1,157 @@ +// 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 ( + "context" + "encoding/json" + "os" + "testing" + "time" + + "github.com/google/go-cmp/cmp" + commonType "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/common/columnselector" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" +) + +type SQLTestHelper struct { + t *testing.T + + helper *commonEvent.EventTestHelper + mounter commonEvent.Mounter + + tableInfo *commonType.TableInfo +} + +func NewSQLTestHelper(t *testing.T, tableName, initialCreateTableDDL string) *SQLTestHelper { + helper := commonEvent.NewEventTestHelperWithTimeZone(t, time.UTC) + helper.Tk().MustExec("set @@tidb_enable_clustered_index=1;") + helper.Tk().MustExec("use test;") + + job := helper.DDL2Job(initialCreateTableDDL) + require.NotNil(t, job) + + mounter := commonEvent.NewMounter(time.UTC) + + tableInfo := helper.GetTableInfo(job) + + return &SQLTestHelper{ + t: t, + helper: helper, + mounter: mounter, + tableInfo: tableInfo, + } +} + +func (h *SQLTestHelper) Close() { + h.helper.Close() +} + +func (h *SQLTestHelper) MustExec(query string, args ...interface{}) { + h.helper.Tk().MustExec(query, args...) +} + +type debeziumSuite struct { + suite.Suite + disableSchema bool +} + +func (s *debeziumSuite) requireDebeziumJSONEq(dbzOutput []byte, tiCDCOutput []byte) { + var ( + ignoredRecordPaths = map[string]bool{ + `{map[string]any}["schema"]`: s.disableSchema, + `{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 + s.Require().Nil(json.Unmarshal(dbzOutput, &objDbzOutput), "Failed to unmarshal Debezium JSON") + + var objTiCDCOutput map[string]any + s.Require().Nil(json.Unmarshal(tiCDCOutput, &objTiCDCOutput), "Failed to unmarshal TiCDC JSON") + + if diff := cmp.Diff(objDbzOutput, objTiCDCOutput, compareOpt); diff != "" { + s.Failf("JSON is not equal", "Diff (-debezium, +ticdc):\n%s", diff) + } +} + +func TestDebeziumSuiteEnableSchema(t *testing.T) { + suite.Run(t, &debeziumSuite{ + disableSchema: false, + }) +} + +func TestDebeziumSuiteDisableSchema(t *testing.T) { + suite.Run(t, &debeziumSuite{ + disableSchema: true, + }) +} + +func (s *debeziumSuite) TestDataTypes() { + dataDDL, err := os.ReadFile("testdata/datatype.ddl.sql") + s.Require().Nil(err) + + dataDML, err := os.ReadFile("testdata/datatype.dml.sql") + s.Require().Nil(err) + + dataDbzOutput, err := os.ReadFile("testdata/datatype.dbz.json") + s.Require().Nil(err) + keyDbzOutput, err := os.ReadFile("testdata/datatype.dbz.key.json") + s.Require().Nil(err) + + helper := NewSQLTestHelper(s.T(), "foo", string(dataDDL)) + + helper.MustExec(`SET sql_mode='';`) + helper.MustExec(`SET time_zone='UTC';`) + dmls := helper.helper.DML2Event("test", "foo", string(dataDML)) + + cfg := common.NewConfig(config.ProtocolDebezium) + cfg.TimeZone = time.UTC + cfg.DebeziumDisableSchema = s.disableSchema + encoder := NewBatchEncoder(cfg, "dbserver1") + for { + row, ok := dmls.GetNextRow() + if !ok { + break + } + err := encoder.AppendRowChangedEvent(context.Background(), "", &commonEvent.RowEvent{ + TableInfo: helper.tableInfo, + CommitTs: 1, + Event: row, + ColumnSelector: columnselector.NewDefaultColumnSelector(), + Callback: func() {}, + }) + s.Require().Nil(err) + } + + messages := encoder.Build() + s.Require().Len(messages, 1) + s.requireDebeziumJSONEq(dataDbzOutput, messages[0].Value) + s.requireDebeziumJSONEq(keyDbzOutput, messages[0].Key) +} diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go index 1bc2ab79a..c0f8c3d07 100644 --- a/pkg/sink/codec/debezium/encoder.go +++ b/pkg/sink/codec/debezium/encoder.go @@ -18,9 +18,11 @@ import ( "context" "time" + "github.com/pingcap/log" commonEvent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/codec/common" + "go.uber.org/zap" ) // BatchEncoder encodes message into Debezium format. @@ -33,35 +35,54 @@ type BatchEncoder struct { // 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 *commonEvent.RowChangedEvent, - callback func(), -) error { + if !d.config.EnableTiDBExtension { + return nil, nil + } + keyMap := bytes.Buffer{} valueBuf := bytes.Buffer{} - err := d.codec.EncodeRowChangedEvent(e, &valueBuf) + err := d.codec.EncodeCheckpointEvent(ts, &keyMap, &valueBuf) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) + } + key, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + keyMap.Bytes(), + ) + if err != nil { + return nil, err } - // TODO: Use a streaming compression is better. value, err := common.Compress( d.config.ChangefeedID, d.config.LargeMessageHandle.LargeMessageHandleCompression, valueBuf.Bytes(), ) if err != nil { + return nil, err + } + result := common.NewMsg(key, value) + return result, nil +} + +// AppendRowChangedEvent implements the RowEventEncoder interface +func (d *BatchEncoder) AppendRowChangedEvent( + _ context.Context, + _ string, + e *commonEvent.RowEvent, +) error { + var key []byte + var value []byte + var err error + if key, err = d.encodeKey(e); err != nil { + return errors.Trace(err) + } + if value, err = d.encodeValue(e); err != nil { return errors.Trace(err) } m := &common.Message{ - Key: nil, + Key: key, Value: value, - Callback: callback, + Callback: e.Callback, } m.IncRowsCount() @@ -72,8 +93,65 @@ func (d *BatchEncoder) AppendRowChangedEvent( // EncodeDDLEvent implements the RowEventEncoder interface // DDL message unresolved tso func (d *BatchEncoder) EncodeDDLEvent(e *commonEvent.DDLEvent) (*common.Message, error) { - // Schema Change Events are currently not supported. - return nil, nil + valueBuf := bytes.Buffer{} + keyMap := bytes.Buffer{} + err := d.codec.EncodeDDLEvent(e, &keyMap, &valueBuf) + if err != nil { + if errors.ErrDDLUnsupportType.Equal(err) { + log.Warn("encode ddl event failed, just ignored", zap.Error(err)) + return nil, nil + } + return nil, errors.Trace(err) + } + key, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + keyMap.Bytes(), + ) + if err != nil { + return nil, err + } + value, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + valueBuf.Bytes(), + ) + if err != nil { + return nil, err + } + result := common.NewMsg(key, value) + + return result, nil +} + +func (d *BatchEncoder) encodeKey(e *commonEvent.RowEvent) ([]byte, error) { + keyBuf := bytes.Buffer{} + err := d.codec.EncodeKey(e, &keyBuf) + if err != nil { + return nil, errors.Trace(err) + } + // TODO: Use a streaming compression is better. + key, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + keyBuf.Bytes(), + ) + return key, err +} + +func (d *BatchEncoder) encodeValue(e *commonEvent.RowEvent) ([]byte, error) { + valueBuf := bytes.Buffer{} + err := d.codec.EncodeValue(e, &valueBuf) + if err != nil { + return nil, errors.Trace(err) + } + // TODO: Use a streaming compression is better. + value, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + valueBuf.Bytes(), + ) + return value, err } // Build implements the RowEventEncoder interface diff --git a/pkg/sink/codec/debezium/helper.go b/pkg/sink/codec/debezium/helper.go new file mode 100644 index 000000000..d1a094a29 --- /dev/null +++ b/pkg/sink/codec/debezium/helper.go @@ -0,0 +1,291 @@ +// 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 ( + "encoding/binary" + "fmt" + "strconv" + "strings" + + "github.com/pingcap/log" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + timodel "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "go.uber.org/zap" +) + +type visiter struct { + columnsMap map[pmodel.CIStr]*timodel.ColumnInfo +} + +func (v *visiter) Enter(n ast.Node) (node ast.Node, skipChildren bool) { + return n, false +} + +func (v *visiter) Leave(n ast.Node) (node ast.Node, ok bool) { + switch col := n.(type) { + case *ast.ColumnDef: + c := v.columnsMap[col.Name.Name] + if col.Tp != nil { + parseType(c, col) + } + c.Comment = "" // disable comment + } + return n, true +} + +func extractValue(expr ast.ExprNode) any { + switch v := expr.(type) { + case *driver.ValueExpr: + return fmt.Sprintf("%v", v.GetValue()) + case *ast.FuncCallExpr: + return v.FnName.String() + } + return nil +} + +func parseType(c *timodel.ColumnInfo, col *ast.ColumnDef) { + ft := col.Tp + switch ft.GetType() { + case mysql.TypeDatetime, mysql.TypeDuration, mysql.TypeTimestamp, mysql.TypeYear: + if ft.GetType() == mysql.TypeYear { + c.SetFlen(ft.GetFlen()) + } else { + c.SetDecimal(ft.GetDecimal()) + } + parseOptions(col.Options, c) + default: + } +} + +func parseOptions(options []*ast.ColumnOption, c *timodel.ColumnInfo) { + for _, option := range options { + switch option.Tp { + case ast.ColumnOptionDefaultValue: + defaultValue := extractValue(option.Expr) + if defaultValue == nil { + continue + } + if err := c.SetDefaultValue(defaultValue); err != nil { + log.Error("failed to set default value") + } + } + } +} + +func parseColumns(sql string, columns []*timodel.ColumnInfo) { + p := parser.New() + stmt, err := p.ParseOneStmt(sql, mysql.DefaultCharset, mysql.DefaultCollationName) + if err != nil { + log.Error("format query parse one stmt failed", zap.Error(err)) + } + + columnsMap := make(map[pmodel.CIStr]*timodel.ColumnInfo, len(columns)) + for _, col := range columns { + columnsMap[col.Name] = col + } + stmt.Accept(&visiter{columnsMap: columnsMap}) +} + +func parseBit(s string, n int) string { + var result string + if len(s) > 0 { + // Leading zeros may be omitted + result = fmt.Sprintf("%0*b", n%8, s[0]) + } + for i := 1; i < len(s); i++ { + result += fmt.Sprintf("%08b", s[i]) + } + return result +} + +func getValueFromDefault(defaultVal any, tp *types.FieldType) any { + // defaultValue shoul be string + // see https://github.com/pingcap/tidb/blob/72b1b7c564c301de33a4bd335a05770c78528db4/pkg/ddl/add_column.go#L791 + val, ok := defaultVal.(string) + if !ok { + log.Debug("default value is not string", zap.Any("defaultValue", defaultVal)) + return defaultVal + } + // TODO: more data types need be consider + switch tp.GetType() { + case mysql.TypeNewDecimal: + return types.NewDecFromStringForTest(val) + case mysql.TypeLonglong, mysql.TypeLong, mysql.TypeInt24, mysql.TypeShort, mysql.TypeTiny, + mysql.TypeYear: + v, err := strconv.ParseInt(val, 10, 64) + if err == nil { + return v + } + log.Error("unexpected column value type string for int column", zap.Error(err), zap.Any("defaultValue", defaultVal)) + case mysql.TypeDouble, mysql.TypeFloat: + v, err := strconv.ParseFloat(val, 64) + if err == nil { + return v + } + log.Error("unexpected column value type string for float column", zap.Error(err), zap.Any("defaultValue", defaultVal)) + } + return defaultVal +} + +func getCharset(ft types.FieldType) string { + if ft.GetCharset() == "binary" { + return "" + } + switch ft.GetType() { + case mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeNewDecimal, mysql.TypeString, mysql.TypeVarchar, + mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeEnum, mysql.TypeSet: + return ft.GetCharset() + } + return "" +} + +func getLen(ft types.FieldType) int { + defaultFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(ft.GetType()) + decimal := ft.GetDecimal() + flen := ft.GetFlen() + switch ft.GetType() { + case mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeDatetime: + return decimal + case mysql.TypeBit, mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTiDBVectorFloat32, + mysql.TypeLonglong, mysql.TypeFloat, mysql.TypeDouble: + if flen != defaultFlen { + return flen + } + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong: + if mysql.HasUnsignedFlag(ft.GetFlag()) { + defaultFlen -= 1 + } + if ft.GetType() == mysql.TypeTiny && mysql.HasZerofillFlag(ft.GetFlag()) { + defaultFlen += 1 + } + if flen != defaultFlen { + return flen + } + case mysql.TypeYear, mysql.TypeNewDecimal: + return flen + case mysql.TypeSet: + return 2*len(ft.GetElems()) - 1 + case mysql.TypeEnum: + return 1 + } + return -1 +} + +func getScale(ft types.FieldType) float64 { + switch ft.GetType() { + case mysql.TypeNewDecimal, mysql.TypeFloat, mysql.TypeDouble: + return float64(ft.GetDecimal()) + } + return -1 +} + +func getSuffix(ft types.FieldType) string { + suffix := "" + decimal := ft.GetDecimal() + flen := ft.GetFlen() + defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(ft.GetType()) + isDecimalNotDefault := decimal != defaultDecimal && decimal != 0 && decimal != -1 + + // displayFlen and displayDecimal are flen and decimal values with `-1` substituted with default value. + displayFlen, displayDecimal := flen, decimal + if displayFlen == -1 { + displayFlen = defaultFlen + } + if displayDecimal == -1 { + displayDecimal = defaultDecimal + } + + switch ft.GetType() { + case mysql.TypeDouble: + // 1. flen Not Default, decimal Not Default -> Valid + // 2. flen Not Default, decimal Default (-1) -> Invalid + // 3. flen Default, decimal Not Default -> Valid + // 4. flen Default, decimal Default -> Valid (hide)W + if isDecimalNotDefault { + suffix = fmt.Sprintf("(%d,%d)", displayFlen, displayDecimal) + } + case mysql.TypeNewDecimal: + suffix = fmt.Sprintf("(%d,%d)", displayFlen, displayDecimal) + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString: + if !mysql.HasBinaryFlag(ft.GetFlag()) && displayFlen != 1 { + suffix = fmt.Sprintf("(%d)", displayFlen) + } + case mysql.TypeYear: + suffix = fmt.Sprintf("(%d)", flen) + case mysql.TypeTiDBVectorFloat32: + if flen != -1 { + suffix = fmt.Sprintf("(%d)", flen) + } + case mysql.TypeNull: + suffix = "(0)" + } + return suffix +} + +func getExpressionAndName(ft types.FieldType) (string, string) { + prefix := strings.ToUpper(types.TypeToStr(ft.GetType(), ft.GetCharset())) + switch ft.GetType() { + case mysql.TypeYear, mysql.TypeBit, mysql.TypeVarchar, mysql.TypeString, mysql.TypeNewDecimal: + return prefix, prefix + } + cs := prefix + getSuffix(ft) + var suf string + if mysql.HasZerofillFlag(ft.GetFlag()) { + suf = " UNSIGNED ZEROFILL" + } else if mysql.HasUnsignedFlag(ft.GetFlag()) { + suf = " UNSIGNED" + } + return cs + suf, prefix + suf +} + +func getTiDBType(ft *types.FieldType) string { + tidbType := types.TypeToStr(ft.GetType(), ft.GetCharset()) + switch ft.GetType() { + case mysql.TypeYear, mysql.TypeBit, mysql.TypeVarchar, mysql.TypeString, mysql.TypeNewDecimal: + return tidbType + } + if mysql.HasUnsignedFlag(ft.GetFlag()) { + tidbType = tidbType + " unsigned" + } + return tidbType +} + +func getBitFromUint64(n int, v uint64) []byte { + var buf [8]byte + binary.LittleEndian.PutUint64(buf[:], v) + numBytes := n / 8 + if n%8 != 0 { + numBytes += 1 + } + return buf[:numBytes] +} + +func getDBTableName(e *commonEvent.DDLEvent) (string, string) { + return e.SchemaName, e.TableName +} + +func getSchemaTopicName(namespace string, schema string, table string) string { + return fmt.Sprintf("%s.%s.%s", + common.SanitizeName(namespace), + common.SanitizeName(schema), + common.SanitizeTopicName(table)) +} diff --git a/pkg/sink/codec/debezium/helper_test.go b/pkg/sink/codec/debezium/helper_test.go new file mode 100644 index 000000000..3bd8461c4 --- /dev/null +++ b/pkg/sink/codec/debezium/helper_test.go @@ -0,0 +1,67 @@ +// 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 ( + "testing" + + timodel "github.com/pingcap/tidb/pkg/meta/model" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" + "github.com/stretchr/testify/require" +) + +func TestGetColumns(t *testing.T) { + sql := "CREATE TABLE test (id INT PRIMARY KEY, val1 datetime default current_timestamp, val2 time(2) default 0, val3 timestamp(3) default now(), val4 YEAR(4) default 1970 comment 'first');" + columnInfos := []*timodel.ColumnInfo{ + { + Name: pmodel.NewCIStr("id"), + FieldType: *types.NewFieldType(mysql.TypeLong), + }, + { + Name: pmodel.NewCIStr("val1"), + FieldType: *types.NewFieldType(mysql.TypeDatetime), + }, + { + Name: pmodel.NewCIStr("val2"), + FieldType: *types.NewFieldType(mysql.TypeDuration), + }, + { + Name: pmodel.NewCIStr("val3"), + FieldType: *types.NewFieldType(mysql.TypeTimestamp), + }, + { + Name: pmodel.NewCIStr("val4"), + FieldType: *types.NewFieldType(mysql.TypeYear), + }, + } + parseColumns(sql, columnInfos) + require.Equal(t, columnInfos[1].GetDefaultValue(), "CURRENT_TIMESTAMP") + require.Equal(t, columnInfos[2].GetDecimal(), 2) + require.Equal(t, columnInfos[2].GetDefaultValue(), "0") + require.Equal(t, columnInfos[3].GetDecimal(), 3) + require.Equal(t, columnInfos[3].GetDefaultValue(), "CURRENT_TIMESTAMP") + require.Equal(t, columnInfos[4].GetFlen(), 4) + require.Equal(t, columnInfos[4].GetDefaultValue(), "1970") + require.Equal(t, columnInfos[4].Comment, "") +} + +func TestGetSchemaTopicName(t *testing.T) { + namespace := "default" + schema := "1A.B" + table := "columnNameWith中文" + name := getSchemaTopicName(namespace, schema, table) + require.Equal(t, name, "default._1A_B.columnNameWith__") +} 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 000000000..0b47f2362 --- /dev/null +++ b/pkg/sink/codec/debezium/testdata/datatype.dbz.json @@ -0,0 +1,651 @@ +{ + "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.Enum", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_enum_invalid" + }, + { + "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.EnumSet", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_set_invalid" + }, + { + "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.Enum", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_enum_invalid" + }, + { + "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.EnumSet", + "version": 1, + "parameters": { "allowed": "a,b,c" }, + "field": "col_set_invalid" + }, + { + "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": "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": 1701673705000, + "snapshot": "false", + "db": "test", + "sequence": null, + "table": "foo", + "server_id": 223344, + "gtid": null, + "file": "mysql-bin.000005", + "pos": 10394, + "row": 0, + "thread": 11, + "query": null + }, + "op": "c", + "ts_ms": 1701673705263, + "transaction": null + } +} diff --git a/pkg/sink/codec/debezium/testdata/datatype.dbz.key.json b/pkg/sink/codec/debezium/testdata/datatype.dbz.key.json new file mode 100644 index 000000000..ce133dbda --- /dev/null +++ b/pkg/sink/codec/debezium/testdata/datatype.dbz.key.json @@ -0,0 +1,17 @@ +{ + "payload": { + "pk": 1 + }, + "schema": { + "fields": [ + { + "field": "pk", + "optional": false, + "type": "int32" + } + ], + "name": "dbserver1.test.foo.Key", + "optional": false, + "type": "struct" + } +} \ No newline at end of file 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 000000000..0381b43ad --- /dev/null +++ b/pkg/sink/codec/debezium/testdata/datatype.ddl.sql @@ -0,0 +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_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 new file mode 100644 index 000000000..b716f80d0 --- /dev/null +++ b/pkg/sink/codec/debezium/testdata/datatype.dml.sql @@ -0,0 +1,69 @@ +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', + 'd', + 'a,b', + 'd', + + '["foo"]' +); diff --git a/pkg/sink/codec/encoder_builder.go b/pkg/sink/codec/encoder_builder.go index 45392c5b3..59ecdbfcd 100644 --- a/pkg/sink/codec/encoder_builder.go +++ b/pkg/sink/codec/encoder_builder.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/ticdc/pkg/sink/codec/canal" "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/ticdc/pkg/sink/codec/csv" + "github.com/pingcap/ticdc/pkg/sink/codec/debezium" "github.com/pingcap/ticdc/pkg/sink/codec/open" "github.com/pingcap/ticdc/pkg/sink/codec/simple" ) @@ -34,8 +35,8 @@ func NewEventEncoder(ctx context.Context, cfg *common.Config) (common.EventEncod return avro.NewAvroEncoder(ctx, cfg) case config.ProtocolCanalJSON: return canal.NewJSONRowEventEncoder(ctx, cfg) - // case config.ProtocolDebezium: - // return debezium.NewBatchEncoder(cfg, config.GetGlobalServerConfig().ClusterID), nil + case config.ProtocolDebezium: + return debezium.NewBatchEncoder(cfg, config.GetGlobalServerConfig().ClusterID), nil case config.ProtocolSimple: return simple.NewEncoder(ctx, cfg) default: diff --git a/tests/integration_tests/debezium/README.md b/tests/integration_tests/debezium/README.md index d57365325..87ef70a30 100644 --- a/tests/integration_tests/debezium/README.md +++ b/tests/integration_tests/debezium/README.md @@ -41,9 +41,9 @@ tiup playground nightly --tiflash 0 --ticdc 1 ``` ``` -tiup cdc cli changefeed create \ - --server=http://127.0.0.1:8300 \ - --sink-uri="kafka://127.0.0.1:9094/output_ticdc?protocol=debezium&kafka-version=2.4.0" +tiup cdc cli changefeed create -c test \ + --server=http://127.0.0.1:8300 --config changefeed.toml \ + --sink-uri="kafka://127.0.0.1:9094/output_ticdc?protocol=debezium" ``` ``` diff --git a/tests/integration_tests/debezium/changefeed.toml b/tests/integration_tests/debezium/changefeed.toml new file mode 100644 index 000000000..995c2d4ed --- /dev/null +++ b/tests/integration_tests/debezium/changefeed.toml @@ -0,0 +1 @@ +force-replicate = true \ No newline at end of file diff --git a/tests/integration_tests/debezium/docker-compose.yml b/tests/integration_tests/debezium/docker-compose.yml index 2041a9800..aa7f09fe3 100644 --- a/tests/integration_tests/debezium/docker-compose.yml +++ b/tests/integration_tests/debezium/docker-compose.yml @@ -45,6 +45,7 @@ services: - CONFIG_STORAGE_TOPIC=my_connect_configs - OFFSET_STORAGE_TOPIC=my_connect_offsets - STATUS_STORAGE_TOPIC=my_connect_statuses + - LANG=C.UTF-8 # watcher_dbz: # For Debug Purpose # restart: always # image: quay.io/debezium/kafka:2.4 diff --git a/tests/integration_tests/debezium/go.mod b/tests/integration_tests/debezium/go.mod index 454c383b0..152279035 100644 --- a/tests/integration_tests/debezium/go.mod +++ b/tests/integration_tests/debezium/go.mod @@ -1,84 +1,82 @@ module github.com/breezewish/checker -go 1.23.0 +go 1.23.4 + +toolchain go1.23.7 require ( + github.com/alecthomas/chroma v0.10.0 + github.com/fatih/color v1.18.0 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/google/go-cmp v0.6.0 + github.com/google/uuid v1.6.0 + github.com/pingcap/tidb v1.1.0-beta.0.20241223052309-3735ed55a394 + github.com/pingcap/tidb/pkg/parser v0.0.0-20241223052309-3735ed55a394 + github.com/segmentio/kafka-go v0.4.45 github.com/thessem/zap-prettyconsole v0.3.0 - go.uber.org/zap v1.26.0 + go.uber.org/zap v1.27.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/cespare/xxhash/v2 v2.3.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/dgryski/go-farm v0.0.0-20240924180020-3414d57e47da // 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/docker/go-units v0.5.0 // indirect + github.com/go-ole/go-ole v1.3.0 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang/protobuf v1.5.3 // indirect + github.com/golang/protobuf v1.5.4 // 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/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.17.9 // 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/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // 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/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f // indirect + github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 // indirect + github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8 // indirect + github.com/pingcap/log v1.1.1-0.20241212030209-7e3ff8601a2a // indirect + github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 // 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/prometheus/client_golang v1.20.5 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.61.0 // indirect + github.com/prometheus/procfs v0.15.1 // 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/shirou/gopsutil/v3 v3.24.5 // 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/tikv/client-go/v2 v2.0.8-0.20241209094930-06d7f4b9233b // indirect + github.com/tikv/pd/client v0.0.0-20241111073742-238d4d79ea31 // 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 + github.com/yusufpapurcu/wmi v1.2.4 // indirect + go.etcd.io/etcd/api/v3 v3.5.12 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.12 // indirect + go.etcd.io/etcd/client/v3 v3.5.12 // 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 + golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 // indirect + golang.org/x/net v0.33.0 // indirect + golang.org/x/sync v0.10.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 // indirect + google.golang.org/grpc v1.63.2 // indirect + google.golang.org/protobuf v1.35.2 // 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 index 3fe1773d2..e4fa4d3ff 100644 --- a/tests/integration_tests/debezium/go.sum +++ b/tests/integration_tests/debezium/go.sum @@ -1,247 +1,152 @@ 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/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c h1:pxW6RcqyfI9/kWtOwnv/G+AzdKuy2ZrqINhenH4HyNs= +github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= 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/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3dyBCFEj5IhUbnKptjxatkF07cF2ak3yi77so= +github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= 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/bits-and-blooms/bitset v1.14.3 h1:Gd2c8lSNf9pKXom5JtD7AaKO8o7fGQ2LtFj1436qilA= +github.com/bits-and-blooms/bitset v1.14.3/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= +github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= +github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= 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/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.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/bbloom v0.0.0-20190830030839-58deb6228d64 h1:W1SHiII3e0jVwvaQFglwu3kS9NLxOeTpvik7MbKCyuQ= +github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64/go.mod h1:F86k/6c7aDUdwSUevnLpHS/3Q9hzYCE99jGk2xsHnt0= github.com/coocood/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/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 h1:NnLfQ77q0G4k2Of2c1ceQ0ec6MkLQyDp+IGdVM0D8XM= +github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= github.com/coreos/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/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= 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/dgryski/go-farm v0.0.0-20240924180020-3414d57e47da h1:aIftn67I1fkbMa512G+w+Pxci9hJPB8oMnkcP3iZF38= +github.com/dgryski/go-farm v0.0.0-20240924180020-3414d57e47da/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/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= +github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= 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/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= +github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= 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/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= +github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= 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-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= 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-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= +github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= 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/glog v1.2.0 h1:uCdmnmatrKCgMBlM4rMuJZWOkPDqdbZPnrMXDY4gI68= +github.com/golang/glog v1.2.0/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= 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/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= 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/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= 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/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/gorilla/mux v1.8.1 h1:TuBL49tXwgrFYWhqrNgrUNEY92u81SPhu7sTdzQEiWY= +github.com/gorilla/mux v1.8.1/go.mod h1:AKf9I4AEqPTmMytcMc0KkNouC66V3BtZ4qD5fmWSiMQ= +github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= +github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= 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/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= 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/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/klauspost/cpuid v1.3.1 h1:5JNjFYYQrZeKRJ0734q51WCEEn2huer72Dc7K+R/b6s= +github.com/klauspost/cpuid v1.3.1/go.mod h1:bYW4mA6ZgKPob1/Dlai2LviZJO7KGI3uoWLd42rAQw4= 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-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/reflect2 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/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= +github.com/ncw/directio v1.0.5 h1:JSUBhdjEvVaJvOoyPAbcW0fnd0tvRXD76wEfZ1KcQz4= +github.com/ncw/directio v1.0.5/go.mod h1:rX/pKEYkOXBGOggmcyJeJGloCkleSvphPx2eV3t6ROk= 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= @@ -249,106 +154,98 @@ github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKw 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/otiai10/copy v1.2.0 h1:HvG945u96iNadPoG2/Ja2+AUJeW5YuFQMixq9yirC+k= +github.com/otiai10/copy v1.2.0/go.mod h1:rrF5dJ5F0t/EWSYODDu4j9/vEeYHMkc8jt0zJChqQWw= +github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7 h1:Dx7Ovyv/SFnMFw3fD4oEoeorXc6saIiQ23LrGLth0Gw= +github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= 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/badger v1.5.1-0.20241015064302-38533b6cbf8d h1:eHcokyHxm7HVM+7+Qy1zZwC7NhX9wVNX8oQDcSZw1qI= +github.com/pingcap/badger v1.5.1-0.20241015064302-38533b6cbf8d/go.mod h1:KiO2zumBCWx7yoVYoFRpb+DNrwEPk1pR1LF7NvOACMQ= 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/errors v0.11.5-0.20240318064555-6bd07397691f h1:FxA+NgsdHNOv+/hZGxUh8Gb3WuZqgqmxDwztEOiA1v4= +github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 h1:tdMsjOqUR7YXHoBitzdebTvOjs/swniBTOLy5XiMtuE= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86/go.mod h1:exzhVYca3WRtd6gclGNErRWb1qEgff3LYta0LvRmON4= 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/kvproto v0.0.0-20241120022153-92b0414aeed8 h1:aNNifhc6xCjXKejjiNYtJJLFNMXnoDiXxkJIg1JErQE= +github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= 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/pingcap/log v1.1.1-0.20241212030209-7e3ff8601a2a h1:WIhmJBlNGmnCWH6TLMdZfNEDaiU8cFpZe3iaqDbQ0M8= +github.com/pingcap/log v1.1.1-0.20241212030209-7e3ff8601a2a/go.mod h1:ORfBOFp1eteu2odzsyaxI+b8TzJwgjwyQcGhI+9SfEA= +github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 h1:T4pXRhBflzDeAhmOQHNPRRogMYxP13V7BkYw3ZsoSfE= +github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5/go.mod h1:rlimy0GcTvjiJqvD5mXTRr8O2eNZPBrcUgiWVYp9530= +github.com/pingcap/tidb v1.1.0-beta.0.20241223052309-3735ed55a394 h1:I7if6LVyQ/hOXiGWP88jZx67x4Lz2YwdaXME9DmaiDo= +github.com/pingcap/tidb v1.1.0-beta.0.20241223052309-3735ed55a394/go.mod h1:HAm8qVL+F0kjr4yp0YWR4cyBdZw0IS53WsE2B0a6GdE= +github.com/pingcap/tidb/pkg/parser v0.0.0-20241223052309-3735ed55a394 h1:Kd5UaT2mbA1gB0G19dkyRB4/Y5DYN8Sy/dgWYg9zNwI= +github.com/pingcap/tidb/pkg/parser v0.0.0-20241223052309-3735ed55a394/go.mod h1:Hju1TEWZvrctQKbztTRwXH7rd41Yq0Pgmq4PrEKcq7o= +github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e h1:7DdrYVwWpYr4o1AyKl8T376B4h2RsMEjkmom8MxQuuM= +github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e/go.mod h1:zrnYy8vReNODg8G0OiYaX9OK+kpq+rK1jHmvd1DnIWw= 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/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/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_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= 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/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.61.0 h1:3gv/GThfX0cV2lpO7gkTUwZru38mxevy90Bj8YFSRQQ= +github.com/prometheus/common v0.61.0/go.mod h1:zr29OCN/2BsJRaFwG8QOBr41D6kkchKbpeNH7pAjb/s= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/qri-io/jsonpointer v0.1.1 h1:prVZBZLL6TW5vsSB9fFHFAMBLI4b0ri5vribQlTJiBA= +github.com/qri-io/jsonpointer v0.1.1/go.mod h1:DnJPaYgiKu56EuDp8TU5wFLdZIcAnb/uH9v37ZaMV64= +github.com/qri-io/jsonschema v0.2.1 h1:NNFoKms+kut6ABPf6xiKNM5214jzxAhDBrPHCJ97Wg0= +github.com/qri-io/jsonschema v0.2.1/go.mod h1:g7DPkiOsK1xv6T/Ao5scXRkd+yTFygcANPBaaqW+VrI= 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/sasha-s/go-deadlock v0.3.5 h1:tNCOEEDG6tBqrNDOX35j/7hL5FcFViG6awUGROb2NsU= +github.com/sasha-s/go-deadlock v0.3.5/go.mod h1:bugP6EGbdGYObIlx7pUZtWqlvo8k9H6vCBBsiChJQ5U= 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/shirou/gopsutil/v3 v3.24.5 h1:i0t8kL+kQTvpAYToeuiVk3TgDeKOFioZO3Ztz/iZ9pI= +github.com/shirou/gopsutil/v3 v3.24.5/go.mod h1:bsoOS1aStSs9ErQ1WWfxllSeS1K5D+U30r2NfcubMVk= 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/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= 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/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= 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/tidwall/btree v1.7.0 h1:L1fkJH/AuEh5zBnnBbmTwQ5Lt+bRJ5A8EWecslvo9iI= +github.com/tidwall/btree v1.7.0/go.mod h1:twD9XRA5jj9VUQGELzDO4HPQTNJsoWWfYEL+EUQ2cKY= +github.com/tikv/client-go/v2 v2.0.8-0.20241209094930-06d7f4b9233b h1:x8E2J8UuUa2ysUkgVfNGgiXxZ9nfqBpQ43PBLwmCitU= +github.com/tikv/client-go/v2 v2.0.8-0.20241209094930-06d7f4b9233b/go.mod h1:NI2GfVlB9n7DsIGCxrKcD4psrcuFNEV8m1BgyzK1Amc= +github.com/tikv/pd/client v0.0.0-20241111073742-238d4d79ea31 h1:oAYc4m5Eu1OY9ogJ103VO47AYPHvhtzbUPD8L8B67Qk= +github.com/tikv/pd/client v0.0.0-20241111073742-238d4d79ea31/go.mod h1:W5a0sDadwUpI9k8p7M77d3jo253ZHdmua+u4Ho4Xw8U= 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= @@ -361,38 +258,25 @@ github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR8 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 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY= github.com/xdg-go/scram v1.1.2/go.mod h1:RT/sEzTbU5y00aCK8UOx6R7YryM0iF1N2MOmC3kKLN4= +github.com/xdg-go/stringprep v1.0.4 h1:XLI/Ng3O1Atzq0oBs3TWm+5ZVgkq2aqdlvP9JtoZ6c8= 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= +github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo0= +github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= +go.etcd.io/etcd/api/v3 v3.5.12 h1:W4sw5ZoU2Juc9gBWuLk5U6fHfNVyY1WC5g9uiXZio/c= +go.etcd.io/etcd/api/v3 v3.5.12/go.mod h1:Ot+o0SWSyT6uHhA56al1oCED0JImsRiU9Dc26+C2a+4= +go.etcd.io/etcd/client/pkg/v3 v3.5.12 h1:EYDL6pWwyOsylrQyLp2w+HkQ46ATiOvoEdMarindU2A= +go.etcd.io/etcd/client/pkg/v3 v3.5.12/go.mod h1:seTzl2d9APP8R5Y2hFL3NVlD6qC/dOT+3kvrqPyTas4= +go.etcd.io/etcd/client/v3 v3.5.12 h1:v5lCPXn1pf1Uu3M4laUE2hp/geOTc5uPcYYsNe1lDxg= +go.etcd.io/etcd/client/v3 v3.5.12/go.mod h1:tSbBCakoWmmddL+BKVAJHa9km+O/E+bumDe9mSbPiqw= 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= @@ -402,8 +286,8 @@ 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/mock v0.4.0 h1:VcM4ZOtdbR4f6VXfiOpwpVJDL6lCReaZ6mw31wqh7KU= +go.uber.org/mock v0.4.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= @@ -413,24 +297,22 @@ 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.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= 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= +go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= +go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= 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/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= +golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= 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/exp v0.0.0-20240909161429-701f63a606c0 h1:e66Fs6Z+fZTbFBAxKfP3PALWBtpfqks2bwGcexMxgtk= +golang.org/x/exp v0.0.0-20240909161429-701f63a606c0/go.mod h1:2TbTHSBQa924w8M6Xs1QcRcFwyucIwBGpK1p2f1YFFY= 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= @@ -442,92 +324,74 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 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/net v0.33.0 h1:74SYHlV8BIgHIFC/LrYkOGIwL19eTYXQ5wc6TBuO36I= +golang.org/x/net v0.33.0/go.mod h1:HXLR5J+9DxmrqMwG9qjGCxZ+zKXxBru04zlTvWlWuN4= 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/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= 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-20211025201205-69cdffdb9359/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.1.0/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/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.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/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= +golang.org/x/time v0.7.0 h1:ntUhktv3OPE6TgYxXWv9vKvUSJyIFJlyohwbkEwPrKQ= +golang.org/x/time v0.7.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= 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= @@ -538,52 +402,37 @@ golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roY 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/tools v0.28.0 h1:WuB6qZ4RPCQo5aP3WdKZS7i595EdWqWR8vqJTlwTVK8= +golang.org/x/tools v0.28.0/go.mod h1:dcIOrVd3mfQKTgrDVQHqCPMWy6lnhfhtX3hLXYVLfRw= 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/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237 h1:RFiFrvy37/mpSpdySBDrUdipW/dHwsRwh3J3+A9VgT4= +google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237/go.mod h1:Z5Iiy3jtmioajWHDGFk7CeugTyHtPvMHA4UTmUkyalE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 h1:AgADTJarZTBqgjiUzRgfaBchgYB3/WFTC80GPwsMcRI= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= 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= +google.golang.org/grpc v1.63.2 h1:MUeiw1B2maTVZthpU5xvASfTh3LDbxHd6IJ6QQVU+xM= +google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= +google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= +google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= 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= @@ -592,10 +441,10 @@ gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWD 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-20200605160147-a5ece683394c/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= diff --git a/tests/integration_tests/debezium/run.sh b/tests/integration_tests/debezium/run.sh index cdffb6cd0..19e8d7665 100644 --- a/tests/integration_tests/debezium/run.sh +++ b/tests/integration_tests/debezium/run.sh @@ -50,7 +50,7 @@ EOF start_tidb_cluster --workdir $WORK_DIR run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY -run_cdc_cli changefeed create --sink-uri="kafka://127.0.0.1:9092/output_ticdc?protocol=debezium&kafka-version=2.4.0" +run_cdc_cli changefeed create -c test --sink-uri="kafka://127.0.0.1:9092/output_ticdc?protocol=debezium&kafka-version=2.4.0" --config "$CUR/changefeed.toml" cd $CUR go run ./src diff --git a/tests/integration_tests/debezium/sql/data_types.sql b/tests/integration_tests/debezium/sql/data_types.sql index 48452db04..534897027 100644 --- a/tests/integration_tests/debezium/sql/data_types.sql +++ b/tests/integration_tests/debezium/sql/data_types.sql @@ -37,8 +37,7 @@ CREATE TABLE t_datetime( col_5 DATETIME(5), col_6 DATETIME(6), col_z DATETIME DEFAULT 0, - -- TODO: - -- col_default_current_timestamp DATETIME DEFAULT CURRENT_TIMESTAMP, + col_default_current_timestamp DATETIME DEFAULT CURRENT_TIMESTAMP, pk INT PRIMARY KEY ); @@ -52,7 +51,7 @@ INSERT INTO t_datetime VALUES ( '2023-11-16 12:34:56.123456', '2023-11-16 12:34:56.123456', NULL, - -- '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', 1 ); @@ -66,7 +65,7 @@ INSERT INTO t_datetime VALUES ( '2023-11-16 12:34:56', '2023-11-16 12:34:56', NULL, - -- '2023-11-16 12:34:56', + '2023-11-16 12:34:56', 2 ); @@ -82,7 +81,7 @@ INSERT INTO t_datetime VALUES ( '2023-11-16 12:34:56.123456', '2023-11-16 12:34:56.123456', NULL, - -- '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', 3 ); @@ -98,7 +97,7 @@ INSERT INTO t_datetime VALUES ( NULL, NULL, NULL, - -- NULL, + NULL, 4 ); @@ -144,27 +143,6 @@ INSERT INTO t_time VALUES ( 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', @@ -183,6 +161,25 @@ INSERT INTO t_time VALUES ( 5 ); +/* + +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', + 6 +); +*/ + /* ---------------------------------------------------------------------- -- TIMESTAMP type diff --git a/tests/integration_tests/debezium/sql/ddl.sql b/tests/integration_tests/debezium/sql/ddl.sql new file mode 100644 index 000000000..071109f7d --- /dev/null +++ b/tests/integration_tests/debezium/sql/ddl.sql @@ -0,0 +1,180 @@ +/* CREATE TABLE */ +CREATE TABLE t1 ( + PK INT PRIMARY KEY, + COL INT +); + +/* CREATE DATABASE */ +CREATE DATABASE foo; +USE foo; + +CREATE TABLE bar ( + PK INT PRIMARY KEY AUTO_INCREMENT, + COL INT +); +INSERT INTO bar VALUES (1, 1); + +/* VIEW */ +CREATE VIEW V1 AS +SELECT * +FROM bar +WHERE COL > 2; +DROP VIEW IF EXISTS V1; + +/* ALTER COLUMN */ +ALTER TABLE bar +ADD COLUMN COL2 INT; +ALTER TABLE bar +MODIFY COLUMN COL2 FLOAT; +ALTER TABLE bar +DROP COLUMN COL2; + +/* Rebase AutoID */ +ALTER TABLE bar AUTO_INCREMENT=310; + +/* Set DEFAULT value */ +ALTER TABLE bar +ALTER COL SET DEFAULT 3; + +/* Modify TABLE comment */ +ALTER TABLE bar COMMENT = 'New table comment'; + +/* Modify TABLE charset */ +ALTER TABLE bar CHARACTER SET = utf8mb4 COLLATE utf8mb4_unicode_ci; + +/* Modify DATABASE charset */ +ALTER DATABASE foo CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci; + +CREATE TABLE t1 ( + PK INT PRIMARY KEY, + COL INT +); + +/* MultiSchemaChange */ +ALTER TABLE test.t1 +CHANGE COL COL2 VARCHAR(255); +ALTER TABLE foo.t1 +CHANGE COL COL2 VARCHAR(255); + +/* PARTITION */ +CREATE TABLE t2 ( + id INT NOT NULL, + year_col INT NOT NULL +) +PARTITION BY RANGE (year_col) ( + PARTITION p0 VALUES LESS THAN (1991), + PARTITION p1 VALUES LESS THAN (1995), + PARTITION p2 VALUES LESS THAN (1999) +); +ALTER TABLE t2 ADD PARTITION (PARTITION p3 VALUES LESS THAN (2002)); +ALTER TABLE t2 REORGANIZE PARTITION p3 INTO ( + PARTITION p31 VALUES LESS THAN (2002), + PARTITION p32 VALUES LESS THAN (2005), + PARTITION p33 VALUES LESS THAN (2008) +); +ALTER TABLE t2 REORGANIZE PARTITION p31,p32,p33,p2 INTO (PARTITION p21 VALUES LESS THAN (2008)); +ALTER TABLE t2 TRUNCATE PARTITION p0; +ALTER TABLE t2 DROP PARTITION p0; +ALTER TABLE t2 PARTITION BY HASH(id) PARTITIONS 10; + +/* ALTER INDEX visibility */ +CREATE TABLE t3 ( + i INT, + j INT, + k INT, + INDEX i_idx (i) INVISIBLE +) ENGINE = InnoDB; +CREATE INDEX j_idx ON t3 (j) INVISIBLE; +ALTER TABLE t3 ADD INDEX k_idx (k) INVISIBLE; +ALTER TABLE t3 ALTER INDEX i_idx VISIBLE; +ALTER TABLE t3 ALTER INDEX i_idx INVISIBLE; + +/* INDEX */ +CREATE TABLE t4 (col1 INT PRIMARY KEY /*T![clustered_index] NONCLUSTERED */, col2 INT); +CREATE INDEX idx1 ON t4 ((col1 + col2)); +CREATE INDEX idx2 ON t4 ((col1 + col2), (col1 - col2), col1); +DROP INDEX idx1 ON t4; +ALTER TABLE t4 ADD INDEX ((col1 * 40) DESC); +ALTER TABLE t4 RENAME INDEX idx2 TO new_idx2; + +/* PRIMARY KEY */ +ALTER TABLE t4 DROP PRIMARY KEY; +ALTER TABLE t4 ADD PRIMARY KEY pk(col1); + +/* + Adding a new column and setting it to the PRIMARY KEY is not supported. + https://docs.pingcap.com/tidb/stable/sql-statement-add-column#mysql-compatibility + ALTER TABLE t4 ADD COLUMN `id` INT(10) primary KEY; +*/ +/* + Dropping primary key columns or columns covered by the composite index is not supported. + https://docs.pingcap.com/tidb/stable/sql-statement-drop-column#mysql-compatibility + ALTER TABLE t4 DROP PRIMARY KEY; +*/ + +/* EXCHANGE PARTITION */ +CREATE TABLE t5 ( + id INT NOT NULL PRIMARY KEY, + fname VARCHAR(30), + lname VARCHAR(30) +) + PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (50), + PARTITION p1 VALUES LESS THAN (100), + PARTITION p2 VALUES LESS THAN (150), + PARTITION p3 VALUES LESS THAN (MAXVALUE) +); +INSERT INTO t5 VALUES (1669, "Jim", "Smith"); +CREATE TABLE t6 LIKE t5; +ALTER TABLE t6 REMOVE PARTITIONING; +ALTER TABLE foo.t5 EXCHANGE PARTITION p0 WITH TABLE foo.t6; + +CREATE TABLE t7 ( + PK INT PRIMARY KEY, + COL INT +); +CREATE TABLE t8 ( + PK INT PRIMARY KEY, + COL INT +); +CREATE TABLE t9 ( + PK INT PRIMARY KEY, + COL INT +); + +/* RENAME TABLE */ +RENAME TABLE t7 TO rename_t7; +RENAME TABLE t8 To rename_t8, t9 To rename_t9; + +/* TRUNCATE TABLE */ +TRUNCATE TABLE rename_t7; + +/* MySQL does not support */ +/* RECOVER TABLE */ +-- DROP TABLE t1; +-- RECOVER TABLE t1; + +/* MySQL does not support */ +/* TTL */ +-- CREATE TABLE t10 ( +-- id int PRIMARY KEY, +-- created_at TIMESTAMP +-- ) /*T![ttl] TTL = `created_at` + INTERVAL 3 MONTH TTL_ENABLE = 'ON'*/; +-- ALTER TABLE t10 TTL = `created_at` + INTERVAL 1 MONTH; +-- ALTER TABLE t10 TTL_ENABLE = 'OFF'; +-- ALTER TABLE t10 REMOVE TTL; + +/* MySQL does not support */ +/* VECTOR INDEX */ +-- CREATE TABLE t11 ( +-- id INT PRIMARY KEY, +-- embedding VECTOR(5) +-- ); +-- ALTER TABLE t11 SET TIFLASH REPLICA 1; +-- CREATE VECTOR INDEX idx_embedding ON t11 ((VEC_COSINE_DISTANCE(embedding))); +-- ALTER TABLE t11 ADD VECTOR INDEX idx_embedding1 ((VEC_L2_DISTANCE(embedding))) USING HNSW; + +/* DROP TABLE */ +DROP TABLE foo.bar; +/* DROP DATABASE */ +DROP DATABASE IF EXISTS foo; diff --git a/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql b/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql index 245452e0e..0925bcfb2 100644 --- a/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: binary_column_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_254_binary_column_test ( id INT AUTO_INCREMENT NOT NULL, file_uuid BINARY(16), diff --git a/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql b/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql index 8ad20ccad..448e55282 100644 --- a/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: binary_column_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_1814_binary_mode_test ( id INT AUTO_INCREMENT NOT NULL, blob_col BLOB NOT NULL, 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 000000000..3b0134985 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql @@ -0,0 +1,15 @@ +SET time_zone = 'UTC'; +-- 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 8ae03647b..82b9564fb 100644 --- a/tests/integration_tests/debezium/sql/debezium/connector_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/connector_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- 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, @@ -12,15 +8,23 @@ CREATE TABLE products ( 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); +VALUES (default,"scooter","Small 2-wheel scooter",3.14); +INSERT INTO products +VALUES (default,"car battery","12V car battery",8.1); +INSERT INTO products +VALUES (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8); +INSERT INTO products +VALUES (default,"hammer","12oz carpenter's hammer",0.75); +INSERT INTO products +VALUES (default,"hammer","14oz carpenter's hammer",0.875); +INSERT INTO products +VALUES (default,"hammer","16oz carpenter's hammer",1.0); +INSERT INTO products +VALUES (default,"rocks","box of assorted rocks",5.3); +INSERT INTO products +VALUES (default,"jacket","water resistent black wind breaker",0.1); +INSERT INTO products +VALUES (default,"spare tire","24 inch spare tire",22.2); -- Create and populate the products on hand using multiple inserts CREATE TABLE products_on_hand ( @@ -49,10 +53,13 @@ CREATE TABLE customers ( 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"); +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"); +INSERT INTO customers +VALUES (default,"George","Bailey","gbailey@foobar.com"); +INSERT INTO customers +VALUES (default,"Edward","Walker","ed@walker.com"); +INSERT INTO customers +VALUES (default,"Anne","Kretchmar","annek@noanswer.org"); -- Create some very simple orders CREATE TABLE orders ( @@ -66,8 +73,12 @@ CREATE TABLE orders ( ) 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); +VALUES (default, '2016-01-16', 1001, 1, 102); +INSERT INTO orders +VALUES (default, '2016-01-17', 1002, 2, 105); +INSERT INTO orders +VALUES (default, '2016-02-18', 1004, 3, 109); +INSERT INTO orders +VALUES (default, '2016-02-19', 1002, 2, 106); +INSERT INTO orders +VALUES (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 index 22ab5d1dd..bf01c6292 100644 --- a/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql +++ b/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: connector_test_ro --- ---------------------------------------------------------------------------------------------------------------- - -- Create and populate our products using a single insert with many rows CREATE TABLE Products ( PRIMARY KEY (id), @@ -13,15 +9,23 @@ CREATE TABLE Products ( 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); +VALUES (default,"scooter","Small 2-wheel scooter",3.14); +INSERT INTO Products +VALUES (default,"car battery","12V car battery",8.1); +INSERT INTO Products +VALUES (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8); +INSERT INTO Products +VALUES (default,"hammer","12oz carpenter's hammer",0.75); +INSERT INTO Products +VALUES (default,"hammer2","14oz carpenter's hammer",8.75E-1); +INSERT INTO Products +VALUES (default,"hammer3","16oz carpenter's hammer",1.0); +INSERT INTO Products +VALUES (default,"rocks","box of assorted rocks",5.3); +INSERT INTO Products +VALUES (default,"jacket","water resistent black wind breaker",0.1); +INSERT INTO Products +VALUES (default,"spare tire","24 inch spare tire",22.2); -- Create and populate the products on hand using multiple inserts CREATE TABLE products_on_hand ( @@ -50,10 +54,13 @@ CREATE TABLE customers ( 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"); +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"); +INSERT INTO customers +VALUES (default,"George","Bailey","gbailey@foobar.com"); +INSERT INTO customers +VALUES (default,"Edward","Walker","ed@walker.com"); +INSERT INTO customers +VALUES (default,"Anne","Kretchmar","annek@noanswer.org"); -- Create some very simple orders CREATE TABLE orders ( @@ -67,11 +74,15 @@ CREATE TABLE orders ( ) 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); +VALUES (default, '2016-01-16', 1001, 1, 102); +INSERT INTO orders +VALUES (default, '2016-01-17', 1002, 2, 105); +INSERT INTO orders +VALUES (default, '2016-02-18', 1004, 3, 109); +INSERT INTO orders +VALUES (default, '2016-02-19', 1002, 2, 106); +INSERT INTO orders +VALUES (default, '2016-02-21', 1003, 1, 107); -- DBZ-342 handle TIME values that exceed the value range of java.sql.Time @@ -83,5 +94,3 @@ CREATE TABLE dbz_342_timetest ( 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 index 08740721b..3ac6788d8 100644 --- a/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: datetime_key_test --- ---------------------------------------------------------------------------------------------------------------- - SET sql_mode=''; CREATE TABLE dbz_1194_datetime_key_test ( id INT AUTO_INCREMENT NOT NULL, diff --git a/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql b/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql index 64cd7742a..4782dc803 100644 --- a/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: decimal_column_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_751_decimal_column_test ( id INT AUTO_INCREMENT NOT NULL, rating1 DECIMAL, diff --git a/tests/integration_tests/debezium/sql/debezium/decimal_test.sql b/tests/integration_tests/debezium/sql/debezium/decimal_test.sql new file mode 100644 index 000000000..a458083d7 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/decimal_test.sql @@ -0,0 +1,10 @@ +CREATE TABLE `DBZ730` ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + A NUMERIC(3, 2) NOT NULL DEFAULT 1.23, + B DECIMAL(4, 3) NOT NULL DEFAULT 2.321, + C NUMERIC(7, 5) NULL DEFAULT '12.678', + D NUMERIC(7, 5) NULL DEFAULT '15.28', + E DECIMAL(65, 18) NULL DEFAULT '0.000000000000000000' +) ENGINE=InnoDB AUTO_INCREMENT=15851 DEFAULT CHARSET=utf8; +INSERT INTO `DBZ730`(A, B, C, D) +VALUES (1.33, -2.111 , 3.444, NULL); diff --git a/tests/integration_tests/debezium/sql/debezium/default_value.sql b/tests/integration_tests/debezium/sql/debezium/default_value.sql new file mode 100644 index 000000000..a9853e68b --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/default_value.sql @@ -0,0 +1,155 @@ +CREATE TABLE UNSIGNED_TINYINT_TABLE ( + id int PRIMARY KEY, + A TINYINT UNSIGNED NULL DEFAULT 0, + B TINYINT UNSIGNED NULL DEFAULT '10', + C TINYINT UNSIGNED NULL, + D TINYINT UNSIGNED NOT NULL, + E TINYINT UNSIGNED NOT NULL DEFAULT 0, + F TINYINT UNSIGNED NOT NULL DEFAULT '0', + G TINYINT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_TINYINT_TABLE VALUES (1, DEFAULT, DEFAULT, 0, 1, DEFAULT, DEFAULT, NULL); + +CREATE TABLE UNSIGNED_SMALLINT_TABLE ( + id int PRIMARY KEY, + A SMALLINT UNSIGNED NULL DEFAULT 0, + B SMALLINT UNSIGNED NULL DEFAULT '10', + C SMALLINT UNSIGNED NULL, + D SMALLINT UNSIGNED NOT NULL, + E SMALLINT UNSIGNED NOT NULL DEFAULT 0, + F SMALLINT UNSIGNED NOT NULL DEFAULT '0', + G SMALLINT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_SMALLINT_TABLE VALUES (1, 1, 1, 1, 0, 1, 1, NULL); + +CREATE TABLE UNSIGNED_MEDIUMINT_TABLE ( + id int PRIMARY KEY, + A MEDIUMINT UNSIGNED NULL DEFAULT 0, + B MEDIUMINT UNSIGNED NULL DEFAULT '10', + C MEDIUMINT UNSIGNED NULL, + D MEDIUMINT UNSIGNED NOT NULL, + E MEDIUMINT UNSIGNED NOT NULL DEFAULT 0, + F MEDIUMINT UNSIGNED NOT NULL DEFAULT '0', + G MEDIUMINT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_MEDIUMINT_TABLE VALUES (1, 1, 1, 1, 0, 1, 1, NULL); + +CREATE TABLE UNSIGNED_INT_TABLE ( + id int PRIMARY KEY, + A INT UNSIGNED NULL DEFAULT 0, + B INT UNSIGNED NULL DEFAULT '10', + C INT UNSIGNED NULL, + D INT UNSIGNED NOT NULL, + E INT UNSIGNED NOT NULL DEFAULT 0, + F INT UNSIGNED NOT NULL DEFAULT '0', + G INT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_INT_TABLE VALUES (1, 1, 1, 1, 0, 1, 1, NULL); + +CREATE TABLE UNSIGNED_BIGINT_TABLE ( + id int PRIMARY KEY, + A BIGINT UNSIGNED NULL DEFAULT 0, + B BIGINT UNSIGNED NULL DEFAULT '10', + C BIGINT UNSIGNED NULL, + D BIGINT UNSIGNED NOT NULL, + E BIGINT UNSIGNED NOT NULL DEFAULT 0, + F BIGINT UNSIGNED NOT NULL DEFAULT '0', + G BIGINT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_BIGINT_TABLE VALUES (1, 1, 1, 1, 0, 1, 1, NULL); + +CREATE TABLE STRING_TABLE ( + id int PRIMARY KEY, + A CHAR(1) NULL DEFAULT 'A', + B CHAR(1) NULL DEFAULT 'b', + C VARCHAR(10) NULL DEFAULT 'CC', + F CHAR(1) DEFAULT NULL, + G VARCHAR(10) DEFAULT NULL, + I VARCHAR(10) NULL DEFAULT '100' +); +INSERT INTO STRING_TABLE +VALUES (1, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, NULL); + +CREATE TABLE BIT_TABLE ( + id int PRIMARY KEY, + A BIT(1) NULL DEFAULT NULL, + B BIT(1) DEFAULT 0, + C BIT(1) DEFAULT 1, + D BIT(1) DEFAULT b'0', + E BIT(1) DEFAULT b'1', + H BIT(10) DEFAULT b'0101000010', + I BIT(10) DEFAULT NULL, + J BIT(25) DEFAULT b'0000000000000000100001111', + K BIT(25) DEFAULT b'0000000010110000100001111' +); +INSERT INTO BIT_TABLE +VALUES (1, false, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, NULL, DEFAULT, NULL); + +CREATE TABLE NUMBER_TABLE ( + id int PRIMARY KEY, + A TINYINT NULL DEFAULT 10, + B SMALLINT NOT NULL DEFAULT '5', + C INTEGER NOT NULL DEFAULT 0, + D BIGINT NOT NULL DEFAULT 20, + E INT NULL DEFAULT NULL, + F INT NULL DEFAULT 30 +); +INSERT INTO NUMBER_TABLE +VALUES (1, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, NULL); + +CREATE TABLE FlOAT_DOUBLE_TABLE ( + id int PRIMARY KEY, + F FLOAT NULL DEFAULT 0, + G DOUBLE NOT NULL DEFAULT 1.1, + H DOUBLE NULL DEFAULT 3.3 +); +INSERT INTO FlOAT_DOUBLE_TABLE +VALUES (1, DEFAULT, DEFAULT, NULL); + +-- set sql_mode REAL_AS_FLOAT is necessary +set @@session.sql_mode=concat(@@session.sql_mode, ',REAL_AS_FLOAT'); +CREATE TABLE REAL_TABLE ( + id int PRIMARY KEY, + A REAL NOT NULL DEFAULT 1, + B REAL NULL DEFAULT NULL, + C REAL NULL DEFAULT 3 +); +INSERT INTO REAL_TABLE +VALUES (1, DEFAULT ,DEFAULT, NULL); + +CREATE TABLE NUMERIC_DECIMAL_TABLE ( + id int PRIMARY KEY, + A NUMERIC(3, 2) NOT NULL DEFAULT 1.23, + B DECIMAL(4, 3) NOT NULL DEFAULT 2.321, + C NUMERIC(7, 5) NULL DEFAULT '12.678', + D NUMERIC(7, 5) NULL DEFAULT '15.28' +); +INSERT INTO NUMERIC_DECIMAL_TABLE +VALUES (1, 1.33 ,2.111 , 3.444, NULL); + +CREATE TABLE DATE_TIME_TABLE ( + id int PRIMARY KEY, + A DATE NOT NULL DEFAULT '1976-08-23', + C DATETIME DEFAULT '2018-01-03 00:00:10', + D DATETIME(1) DEFAULT '2018-01-03 00:00:10.7', + E DATETIME(6) DEFAULT '2018-01-03 00:00:10.123456', + F YEAR NOT NULL DEFAULT 1, + G TIME DEFAULT '00:00:00', + H TIME(1) DEFAULT '23:00:00.7', + I TIME(6) DEFAULT '23:00:00.123456', + J TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + K TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP, + L TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP, + M TIME(1) DEFAULT '-23:45:56.7', + N TIME(6) DEFAULT '123:00:00.123456' +); +INSERT INTO DATE_TIME_TABLE +VALUES (1, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, NULL, DEFAULT, DEFAULT, DEFAULT); + +CREATE TABLE DBZ_771_CUSTOMERS ( + id INTEGER NOT NULL PRIMARY KEY, + CUSTOMER_TYPE ENUM ('b2c','b2b') NOT NULL default 'b2c' +); + +INSERT INTO DBZ_771_CUSTOMERS +VALUES (1, 'b2b'); \ No newline at end of file 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 000000000..d6eba9d57 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql @@ -0,0 +1,33 @@ +SET sql_mode=''; +CREATE TABLE all_zero_date_and_time_table ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + 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 +); +INSERT INTO all_zero_date_and_time_table +VALUES (DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT); + +CREATE TABLE part_zero_date_and_time_table ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + 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', + /* TiCDC convert '1000-00-00 01:00:00.000' to "1000-00-00" + 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 +); +INSERT INTO part_zero_date_and_time_table +VALUES (DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, /*DEFAULT,*/ DEFAULT, DEFAULT, DEFAULT); \ No newline at end of file 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 000000000..ed4841620 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/default_value_generated.sql @@ -0,0 +1,8 @@ +/* see https://github.com/pingcap/tiflow/issues/11704 */ +CREATE TABLE GENERATED_TABLE ( + id int PRIMARY KEY, + A SMALLINT UNSIGNED, + B SMALLINT UNSIGNED AS (2 * A) STORED, + C SMALLINT UNSIGNED AS (3 * A) STORED NOT NULL +); +INSERT INTO GENERATED_TABLE VALUES (1, 15, DEFAULT, DEFAULT); diff --git a/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql b/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql index a8db5dfe3..9c261cbf4 100644 --- a/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql @@ -1,7 +1,9 @@ +SET time_zone='UTC'; +/* Debezium convert charsetName to "utf8mb4" when column collate is utf8_unicode_ci */ 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', + `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`) @@ -9,8 +11,9 @@ CREATE TABLE `test_stations_10` ( 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'; +/* Debezium datetime length is not correct */ +-- 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' ); +-- INSERT INTO test_stations_10 ( `name`, `type` ) values ( 'Ha Tinh 1', 'now' ); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/float_test.sql b/tests/integration_tests/debezium/sql/debezium/float_test.sql new file mode 100644 index 000000000..da2d3bbf0 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/float_test.sql @@ -0,0 +1,22 @@ +/* + TiDB uses this value only to determine whether to use FLOAT or DOUBLE for the resulting data type. + If p is from 0 to 24, the data type becomes FLOAT with no M or D values. + If p is from 25 to 53, the data type becomes DOUBLE with no M or D values. +*/ +SET sql_mode=''; +CREATE TABLE `DBZ3865` ( + `id` INT NOT NULL AUTO_INCREMENT, + `f1` FLOAT DEFAULT 5.6, + `f2` FLOAT(10, 2) DEFAULT NULL, + `f3` FLOAT(35, 5) DEFAULT NULL, + /* TiDB incorrect length output. issue:https://github.com/pingcap/tidb/issues/57060 + `f4_23` FLOAT(23) DEFAULT NULL + `f4_24` FLOAT(24) DEFAULT NULL, + `f4_25` FLOAT(25) DEFAULT NULL, */ + `weight` FLOAT UNSIGNED DEFAULT '0', + PRIMARY KEY (`ID`) +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8mb4; +/* Debezium incorrect output: + f2: 5.610000133514404 + f3: 30.12346076965332 */ +INSERT INTO DBZ3865(f1,/* f2, f3, f4_23, f4_24,*/ weight) VALUE (5.6,/* 5.61, 30.123456, 64.1, 64.1,*/ 64.1234); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/history-dbz.sql b/tests/integration_tests/debezium/sql/debezium/history-dbz.sql new file mode 100644 index 000000000..5a328e451 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/history-dbz.sql @@ -0,0 +1,2 @@ +CREATE TABLE `t-1` (ID1 INT PRIMARY KEY); +CREATE TABLE `t.2` (ID2 INT PRIMARY KEY); diff --git a/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql b/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql index 679bb0169..4d805f52c 100644 --- a/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql +++ b/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql @@ -1,10 +1,9 @@ --- ---------------------------------------------------------------------------------------------------------------- --- 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 +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE t3; +DROP TABLE t4; \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-123.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-123.ddl new file mode 100644 index 000000000..b09b625a0 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-123.ddl @@ -0,0 +1,12 @@ +CREATE TABLE `DBZ123` ( + `Id` bigint(20) NOT NULL AUTO_INCREMENT, + `Provider_ID` bigint(20) NOT NULL, + `External_ID` varchar(255) NOT NULL, + `Name` varchar(255) NOT NULL, + `Is_Enabled` bit(1) NOT NULL DEFAULT b'1', + /* BLOB, TEXT, GEOMETRY or JSON column 'binaryRepresentation' can't have a default value + `binaryRepresentation` BLOB NOT NULL DEFAULT x'cafe', */ + `BonusFactor` decimal(19,8) NOT NULL, + PRIMARY KEY (`Id`), + UNIQUE KEY `game_unq` (`Provider_ID`,`External_ID`) +) ENGINE=InnoDB AUTO_INCREMENT=2374 DEFAULT CHARSET=utf8 \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-162.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-162.ddl new file mode 100644 index 000000000..caa4d4ae4 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-162.ddl @@ -0,0 +1,7 @@ +CREATE TABLE `test` (id INT(11) UNSIGNED NOT NULL PRIMARY KEY AUTO_INCREMENT); + +ALTER TABLE `test` CHANGE `id` `collection_id` INT(11) +UNSIGNED +NOT NULL +AUTO_INCREMENT; + diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-193.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-193.ddl new file mode 100644 index 000000000..0d3026a4c --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-193.ddl @@ -0,0 +1,10 @@ +CREATE TABLE `roles` ( +`id` varchar(32) NOT NULL, +`name` varchar(100) NOT NULL, +`context` varchar(20) NOT NULL, +`organization_id` int(11) DEFAULT NULL, +`client_id` varchar(32) NOT NULL, +`scope_action_ids` text NOT NULL, +PRIMARY KEY (`id`), +FULLTEXT KEY `scope_action_ids_idx` (`scope_action_ids`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-198i.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-198i.ddl new file mode 100644 index 000000000..36eb69a9d --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-198i.ddl @@ -0,0 +1,23 @@ +create database `NextTimeTable`; +use `NextTimeTable`; +create table NextTimeTable.REFERENCED ( + SUBJECT_ID int not null, + PRIMARY KEY (SUBJECT_ID) + ); + +create table `NextTimeTable`.`TIMETABLE_SUBJECT_GROUP_MAPPING` ( + pk1 int not null, + `SUBJECT_ID` int not null, + `other` int, + CONSTRAINT `FK69atxmt7wrwpb4oekyravsx9l` FOREIGN KEY (`SUBJECT_ID`) REFERENCES `NextTimeTable`.`REFERENCED`(`SUBJECT_ID`) + ); + +/* TiCDC discards it due to unsupported DDL type. +Alter table `NextTimeTable`.`TIMETABLE_SUBJECT_GROUP_MAPPING` +drop foreign key `FK69atxmt7wrwpb4oekyravsx9l`; +Alter table `NextTimeTable`.`TIMETABLE_SUBJECT_GROUP_MAPPING` +drop index `FK69atxmt7wrwpb4oekyravsx9l`; +Alter table `NextTimeTable`.`TIMETABLE_SUBJECT_GROUP_MAPPING` +drop column `SUBJECT_ID`; +*/ +create table `NextTimeTable`.`table1` ( pk1 int not null PRIMARY KEY, `id` int not null, `other` int ); diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-200.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-200.ddl new file mode 100644 index 000000000..797ed7304 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-200.ddl @@ -0,0 +1,25 @@ +CREATE TABLE `customfield` ( + `ENCODEDKEY` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + `ID` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `CREATIONDATE` datetime DEFAULT NULL, + `LASTMODIFIEDDATE` datetime DEFAULT NULL, + `DATATYPE` varchar(256) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `ISDEFAULT` bit(1) DEFAULT NULL, + `ISREQUIRED` bit(1) DEFAULT NULL, + `NAME` varchar(256) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `VALUES` mediumblob, + `AMOUNTS` mediumblob, + `DESCRIPTION` varchar(256) DEFAULT NULL, + `TYPE` varchar(256) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `VALUELENGTH` varchar(256) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL DEFAULT 'SHORT', + `INDEXINLIST` int(11) DEFAULT '-1', + `CUSTOMFIELDSET_ENCODEDKEY_OID` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + `STATE` varchar(256) NOT NULL DEFAULT 'NORMAL', + `VALIDATIONPATTERN` varchar(256) DEFAULT NULL, + `VIEWUSAGERIGHTSKEY` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `EDITUSAGERIGHTSKEY` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `BUILTINCUSTOMFIELDID` varchar(255) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `UNIQUE` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + STORAGE varchar(32) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + KEY `index1` (`ID`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-quoted.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-quoted.ddl new file mode 100644 index 000000000..cad5ecae4 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-quoted.ddl @@ -0,0 +1,36 @@ +-- +-- Contains statements with quoted identifiers +-- +CREATE TABLE `customers` ( + `id` int(11) NOT NULL AUTO_INCREMENT, + `first_name` varchar(255) NOT NULL, + `last_name` varchar(255) NOT NULL, + `email` varchar(255) NOT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `email` (`email`) +) ENGINE=InnoDB AUTO_INCREMENT=1005 DEFAULT CHARSET=latin1; +CREATE TABLE `products` ( + `id` int(11) NOT NULL AUTO_INCREMENT, + `name` varchar(255) NOT NULL, + `description` varchar(512) DEFAULT NULL, + `weight` float DEFAULT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=110 DEFAULT CHARSET=latin1; +CREATE TABLE `orders` ( + `order_number` int(11) NOT NULL AUTO_INCREMENT, + `order_date` date NOT NULL, + `purchaser` int(11) NOT NULL, + `quantity` int(11) NOT NULL, + `product_id` int(11) NOT NULL, + PRIMARY KEY (`order_number`), + KEY `order_customer` (`purchaser`), + KEY `ordered_product` (`product_id`), + CONSTRAINT `orders_ibfk_1` FOREIGN KEY (`purchaser`) REFERENCES `customers` (`id`), + CONSTRAINT `orders_ibfk_2` FOREIGN KEY (`product_id`) REFERENCES `products` (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=10006 DEFAULT CHARSET=latin1; +CREATE TABLE `products_on_hand` ( + `product_id` int(11) NOT NULL, + `quantity` int(11) NOT NULL, + PRIMARY KEY (`product_id`), + CONSTRAINT `products_on_hand_ibfk_1` FOREIGN KEY (`product_id`) REFERENCES `products` (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=latin1 \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-test-create.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-test-create.ddl new file mode 100644 index 000000000..705270899 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-test-create.ddl @@ -0,0 +1,1134 @@ +-- +-- BUILD SCRIPT +-- RDBMS: MYSQL 5.0 +-- + +CREATE TABLE AUDITENTRIES +( + TIMESTAMP VARCHAR(50) NOT NULL, + CONTEXT VARCHAR(64) NOT NULL, + ACTIVITY VARCHAR(64) NOT NULL, + RESOURCES VARCHAR(4000) NOT NULL, + PRINCIPAL VARCHAR(255) NOT NULL, + HOSTNAME VARCHAR(64) NOT NULL, + VMID VARCHAR(64) NOT NULL +); + +CREATE TABLE AUTHPERMTYPES +( + PERMTYPEUID NUMERIC(10) NOT NULL PRIMARY KEY, + DISPLAYNAME VARCHAR(250) NOT NULL, + FACTORYCLASSNAME VARCHAR(80) NOT NULL +); + +CREATE TABLE AUTHPOLICIES +( + POLICYUID NUMERIC(10) NOT NULL PRIMARY KEY, + DESCRIPTION VARCHAR(250), + POLICYNAME VARCHAR(250) NOT NULL +); + +CREATE TABLE AUTHPRINCIPALS +( + PRINCIPALTYPE NUMERIC(10) NOT NULL, + PRINCIPALNAME VARCHAR(255) NOT NULL, + POLICYUID NUMERIC(10) NOT NULL REFERENCES AUTHPOLICIES (POLICYUID) , + GRANTOR VARCHAR(255) NOT NULL, + CONSTRAINT PK_AUTHPOLICYPRINCIPALS UNIQUE (PRINCIPALNAME, POLICYUID) +); + +CREATE TABLE AUTHREALMS +( + REALMUID NUMERIC(10) NOT NULL PRIMARY KEY, + REALMNAME VARCHAR(250) NOT NULL UNIQUE, + DESCRIPTION VARCHAR(550) +); + +CREATE TABLE CFG_STARTUP_STATE +(STATE INTEGER DEFAULT 0 , +LASTCHANGED VARCHAR(50) ); + +CREATE TABLE IDTABLE +( + IDCONTEXT VARCHAR(20) NOT NULL PRIMARY KEY, + NEXTID NUMERIC +); + +CREATE TABLE LOGMESSAGETYPES +( + MESSAGELEVEL NUMERIC(10) NOT NULL PRIMARY KEY, + NAME VARCHAR(64) NOT NULL, + DISPLAYNAME VARCHAR(64) +); + +CREATE TABLE MM_PRODUCTS +( + PRODUCT_UID NUMERIC NOT NULL PRIMARY KEY, + PRODUCT_NAME VARCHAR(50) NOT NULL, + PRODUCT_DISPLAY_NM VARCHAR(100) +); + +CREATE TABLE PRINCIPALTYPES +( + PRINCIPALTYPEUID NUMERIC(10) NOT NULL PRIMARY KEY, + PRINCIPALTYPE VARCHAR(60) NOT NULL, + DISPLAYNAME VARCHAR(80) NOT NULL, + LASTCHANGEDBY VARCHAR(255) NOT NULL, + LASTCHANGED VARCHAR(50) +); +-- ========= STATEMENT 10 ============ + +CREATE TABLE RT_MDLS +( + MDL_UID NUMERIC(10) NOT NULL PRIMARY KEY, + MDL_UUID VARCHAR(64) NOT NULL, + MDL_NM VARCHAR(255) NOT NULL, + MDL_VERSION VARCHAR(50), + DESCRIPTION VARCHAR(255), + MDL_URI VARCHAR(255), + MDL_TYPE NUMERIC(3), + IS_PHYSICAL CHAR(1) NOT NULL, + MULTI_SOURCED CHAR(1) DEFAULT '0', + VISIBILITY NUMERIC(10) + ); + +CREATE TABLE RT_MDL_PRP_NMS +( + PRP_UID NUMERIC(10) NOT NULL PRIMARY KEY, + MDL_UID NUMERIC(10) NOT NULL , + PRP_NM VARCHAR(255) NOT NULL +); + +CREATE TABLE RT_MDL_PRP_VLS +( + PRP_UID NUMERIC(10) NOT NULL , + PART_ID NUMERIC(10) NOT NULL, + PRP_VL VARCHAR(255) NOT NULL, + CONSTRAINT PK_MDL_PRP_VLS UNIQUE (PRP_UID, PART_ID) +); + + +CREATE TABLE RT_VIRTUAL_DBS +( + VDB_UID NUMERIC(10) NOT NULL PRIMARY KEY, + VDB_VERSION VARCHAR(50) NOT NULL, + VDB_NM VARCHAR(255) NOT NULL, + DESCRIPTION VARCHAR(255), + PROJECT_GUID VARCHAR(64), + VDB_STATUS NUMERIC NOT NULL, + WSDL_DEFINED CHAR(1) DEFAULT '0', + VERSION_BY VARCHAR(100), + VERSION_DATE VARCHAR(50) NOT NULL, + CREATED_BY VARCHAR(100), + CREATION_DATE VARCHAR(50), + UPDATED_BY VARCHAR(100), + UPDATED_DATE VARCHAR(50), + VDB_FILE_NM VARCHAR(2048) +); + +CREATE TABLE SERVICESESSIONS +( + SESSIONUID NUMERIC(10) NOT NULL PRIMARY KEY, + PRINCIPAL VARCHAR(255) NOT NULL, + APPLICATION VARCHAR(128) NOT NULL, + CREATIONTIME VARCHAR(50), + CLIENTCOUNT NUMERIC(10) NOT NULL, + STATE NUMERIC(10) NOT NULL, + STATETIME VARCHAR(50), + USESSUBSCRIBER CHAR(1) NOT NULL, + PRODUCTINFO1 VARCHAR(255), + PRODUCTINFO2 VARCHAR(255), + PRODUCTINFO3 VARCHAR(255), + PRODUCTINFO4 VARCHAR(255) +); +-- ========= STATEMENT 15 ============ +CREATE INDEX RTMDLS_NM_IX ON RT_MDLS (MDL_NM); + +CREATE INDEX RTVIRTUALDBS_NM_IX ON RT_VIRTUAL_DBS (VDB_NM); + +CREATE INDEX RTVIRTUALDBS_VRSN_IX ON RT_VIRTUAL_DBS (VDB_VERSION); + +CREATE UNIQUE INDEX MDL_PRP_NMS_UIX ON RT_MDL_PRP_NMS (MDL_UID, PRP_NM); + +CREATE UNIQUE INDEX PRNCIPALTYP_UIX ON PRINCIPALTYPES (PRINCIPALTYPE); +-- ========= STATEMENT 20 ============ +CREATE UNIQUE INDEX AUTHPOLICIES_NAM_UIX ON AUTHPOLICIES (POLICYNAME); + +CREATE TABLE AUTHPERMISSIONS +( + PERMISSIONUID NUMERIC(10) NOT NULL PRIMARY KEY, + RESOURCENAME VARCHAR(250) NOT NULL, + ACTIONS NUMERIC(10) NOT NULL, + CONTENTMODIFIER VARCHAR(250), + PERMTYPEUID NUMERIC(10) NOT NULL REFERENCES AUTHPERMTYPES (PERMTYPEUID) , + REALMUID NUMERIC(10) NOT NULL REFERENCES AUTHREALMS (REALMUID) , + POLICYUID NUMERIC(10) NOT NULL REFERENCES AUTHPOLICIES (POLICYUID) +); + + +CREATE TABLE LOGENTRIES +( + TIMESTAMP VARCHAR(50) NOT NULL, + CONTEXT VARCHAR(64) NOT NULL, + MSGLEVEL NUMERIC(10) NOT NULL REFERENCES LOGMESSAGETYPES (MESSAGELEVEL) , + EXCEPTION VARCHAR(4000), + MESSAGE VARCHAR(2000) NOT NULL, + HOSTNAME VARCHAR(64) NOT NULL, + VMID VARCHAR(64) NOT NULL, + THREAModeShapeME VARCHAR(64) NOT NULL, + VMSEQNUM NUMERIC(7) NOT NULL +); + +CREATE TABLE PRODUCTSSESSIONS +( + PRODUCT_UID NUMERIC NOT NULL, + SESSION_UID NUMERIC NOT NULL, + PRIMARY KEY (PRODUCT_UID, SESSION_UID) +); + +-- ALTER TABLE PRODUCTSSESSIONS +-- ADD CONSTRAINT FK_PRODSESS_PRODS +-- FOREIGN KEY (PRODUCT_UID) +-- REFERENCES MM_PRODUCTS (PRODUCT_UID); + +-- ALTER TABLE PRODUCTSSESSIONS +-- ADD CONSTRAINT FK_PRODSESS_SVCSES +-- FOREIGN KEY (SESSION_UID) +-- REFERENCES SERVICESESSIONS (SESSIONUID); + + +CREATE TABLE RT_VDB_MDLS +( + VDB_UID NUMERIC(10) NOT NULL , + MDL_UID NUMERIC(10) NOT NULL , + CNCTR_BNDNG_NM VARCHAR(255) +); + +CREATE INDEX AWA_SYS_MSGLEVEL_1E6F845E ON LOGENTRIES (MSGLEVEL); + +CREATE UNIQUE INDEX AUTHPERM_UIX ON AUTHPERMISSIONS ( POLICYUID, RESOURCENAME); + +CREATE TABLE CS_EXT_FILES ( + FILE_UID INTEGER NOT NULL, + CHKSUM NUMERIC(20), + FILE_NAME VARCHAR(255) NOT NULL, + FILE_CONTENTS LONGBLOB, + CONFIG_CONTENTS LONGTEXT, + SEARCH_POS INTEGER, + IS_ENABLED CHAR(1), + FILE_DESC VARCHAR(4000), + CREATED_BY VARCHAR(100), + CREATION_DATE VARCHAR(50), + UPDATED_BY VARCHAR(100), + UPDATE_DATE VARCHAR(50), + FILE_TYPE VARCHAR(30), + CONSTRAINT PK_CS_EXT_FILES PRIMARY KEY (FILE_UID) +) +; +-- ========= STATEMENT 30 ============ +-- ALTER TABLE CS_EXT_FILES ADD CONSTRAINT CSEXFILS_FIL_NA_UK UNIQUE (FILE_NAME); + +CREATE TABLE MMSCHEMAINFO_CA +( + SCRIPTNAME VARCHAR(50), + SCRIPTEXECUTEDBY VARCHAR(50), + SCRIPTREV VARCHAR(50), + RELEASEDATE VARCHAR(50), + DATECREATED DATE, + DATEUPDATED DATE, + UPDATEID VARCHAR(50), + METAMATRIXSERVERURL VARCHAR(100) +) +; + +CREATE TABLE CS_SYSTEM_PROPS ( + PROPERTY_NAME VARCHAR(255), + PROPERTY_VALUE VARCHAR(255) +); + +CREATE UNIQUE INDEX SYSPROPS_KEY ON CS_SYSTEM_PROPS (PROPERTY_NAME); + +CREATE TABLE CFG_LOCK ( + USER_NAME VARCHAR(50) NOT NULL, + DATETIME_ACQUIRED VARCHAR(50) NOT NULL, + DATETIME_EXPIRE VARCHAR(50) NOT NULL, + HOST VARCHAR(100), + LOCK_TYPE NUMERIC (1) ); + + +CREATE TABLE TX_MMXCMDLOG +(REQUESTID VARCHAR(255) NOT NULL, +TXNUID VARCHAR(50) NULL, +CMDPOINT NUMERIC(10) NOT NULL, +SESSIONUID VARCHAR(255) NOT NULL, +APP_NAME VARCHAR(255) NULL, +PRINCIPAL_NA VARCHAR(255) NOT NULL, +VDBNAME VARCHAR(255) NOT NULL, +VDBVERSION VARCHAR(50) NOT NULL, +CREATED_TS VARCHAR(50) NULL, +ENDED_TS VARCHAR(50) NULL, +CMD_STATUS NUMERIC(10) NOT NULL, +SQL_ID NUMERIC(10), +FINL_ROWCNT NUMERIC(10) +) +; + +CREATE TABLE TX_SRCCMDLOG +(REQUESTID VARCHAR(255) NOT NULL, +NODEID NUMERIC(10) NOT NULL, +SUBTXNUID VARCHAR(50) NULL, +CMD_STATUS NUMERIC(10) NOT NULL, +MDL_NM VARCHAR(255) NOT NULL, +CNCTRNAME VARCHAR(255) NOT NULL, +CMDPOINT NUMERIC(10) NOT NULL, +SESSIONUID VARCHAR(255) NOT NULL, +PRINCIPAL_NA VARCHAR(255) NOT NULL, +CREATED_TS VARCHAR(50) NULL, +ENDED_TS VARCHAR(50) NULL, +SQL_ID NUMERIC(10) NULL, +FINL_ROWCNT NUMERIC(10) NULL +) +; + + +CREATE TABLE TX_SQL ( SQL_ID NUMERIC(10) NOT NULL, + SQL_VL TEXT ) +; +ALTER TABLE TX_SQL + ADD CONSTRAINT TX_SQL_PK +PRIMARY KEY (SQL_ID) +; +-- ========= STATEMENT 39 ============ + +-- +-- The ITEMS table stores the raw, structure-independent information about the items contained by the Repository. This table is capable of persisting multiple versions of an item. +-- +CREATE TABLE MBR_ITEMS +( + ITEM_ID_P1 NUMERIC(20) NOT NULL, + ITEM_ID_P2 NUMERIC(20) NOT NULL, + ITEM_VERSION VARCHAR(80) NOT NULL, + ITEM_NAME VARCHAR(255) NOT NULL, + UPPER_ITEM_NAME VARCHAR(255) NOT NULL, + COMMENT_FLD VARCHAR(2000), + LOCK_HOLDER VARCHAR(100), + LOCK_DATE VARCHAR(50), + CREATED_BY VARCHAR(100) NOT NULL, + CREATION_DATE VARCHAR(50) NOT NULL, + ITEM_TYPE NUMERIC(10) NOT NULL +); + +-- +-- The ITEM_CONTENTS table stores the contents for items (files) stored in the repository. This table is capable of persisting multiple versions of the contents for an item. +-- +CREATE TABLE MBR_ITEM_CONTENTS +( + ITEM_ID_P1 NUMERIC(20) NOT NULL, + ITEM_ID_P2 NUMERIC(20) NOT NULL, + ITEM_VERSION VARCHAR(80) NOT NULL, + ITEM_CONTENT LONGBLOB NOT NULL +); + +-- +-- The ENTRIES table stores the structure information for all the objects stored in the Repository. This includes both folders and items. +-- +CREATE TABLE MBR_ENTRIES +( + ENTRY_ID_P1 NUMERIC(20) NOT NULL, + ENTRY_ID_P2 NUMERIC(20) NOT NULL, + ENTRY_NAME VARCHAR(255) NOT NULL, + UPPER_ENTRY_NAME VARCHAR(255) NOT NULL, + ITEM_ID_P1 NUMERIC(20), + ITEM_ID_P2 NUMERIC(20), + ITEM_VERSION VARCHAR(80), + PARENT_ENTRY_ID_P1 NUMERIC(20), + PARENT_ENTRY_ID_P2 NUMERIC(20), + DELETED NUMERIC(1) NOT NULL +); + +-- +-- The LABELS table stores the various labels that have been defined. +-- +CREATE TABLE MBR_LABELS +( + LABEL_ID_P1 NUMERIC(20) NOT NULL, + LABEL_ID_P2 NUMERIC(20) NOT NULL, + LABEL_FLD VARCHAR(255) NOT NULL, + COMMENT_FLD VARCHAR(2000), + CREATED_BY VARCHAR(100) NOT NULL, + CREATION_DATE VARCHAR(50) NOT NULL +); + +-- +-- The ITEM_LABELS table maintains the relationships between the ITEMS and the LABELs; that is, the labels that have been applied to each of the item versions. (This is a simple intersect table.) +-- +CREATE TABLE MBR_ITEM_LABELS +( + ITEM_ID_P1 NUMERIC(20) NOT NULL, + ITEM_ID_P2 NUMERIC(20) NOT NULL, + ITEM_VERSION VARCHAR(80) NOT NULL, + LABEL_ID_P1 NUMERIC(20) NOT NULL, + LABEL_ID_P2 NUMERIC(20) NOT NULL +); + +-- +-- The ITEM_LABELS table maintains the relationships between the ITEMS and the LABELs; that is, the labels that have been applied to each of the item versions. (This is a simple intersect table.) +-- +CREATE TABLE MBR_FOLDER_LABELS +( + ENTRY_ID_P1 NUMERIC(20) NOT NULL, + ENTRY_ID_P2 NUMERIC(20) NOT NULL, + LABEL_ID_P1 NUMERIC(20) NOT NULL, + LABEL_ID_P2 NUMERIC(20) NOT NULL +); + +CREATE TABLE MBR_ITEM_TYPES +( + ITEM_TYPE_CODE NUMERIC(10) NOT NULL, + ITEM_TYPE_NM VARCHAR(20) NOT NULL +); + +CREATE TABLE MBR_POLICIES +( + POLICY_NAME VARCHAR(250) NOT NULL, + CREATION_DATE VARCHAR(50), + CHANGE_DATE VARCHAR(50), + GRANTOR VARCHAR(32) +); + +CREATE TABLE MBR_POL_PERMS +( + ENTRY_ID_P1 NUMERIC(20) NOT NULL, + ENTRY_ID_P2 NUMERIC(20) NOT NULL, + POLICY_NAME VARCHAR(250) NOT NULL, + CREATE_BIT CHAR(1) NOT NULL, + READ_BIT CHAR(1) NOT NULL, + UPDATE_BIT CHAR(1) NOT NULL, + DELETE_BIT CHAR(1) NOT NULL +); + +CREATE TABLE MBR_POL_USERS +( + POLICY_NAME VARCHAR(250) NOT NULL, + USER_NAME VARCHAR(80) NOT NULL +); + +CREATE UNIQUE INDEX MBR_ENT_NM_PNT_IX ON MBR_ENTRIES (UPPER_ENTRY_NAME,PARENT_ENTRY_ID_P1,PARENT_ENTRY_ID_P2); +-- ========= STATEMENT 50 ============ +CREATE INDEX MBR_ITEMS_ID_IX ON MBR_ITEMS (ITEM_ID_P1,ITEM_ID_P2); + +CREATE INDEX MBR_ENT_PARNT_IX ON MBR_ENTRIES (PARENT_ENTRY_ID_P1); + +CREATE INDEX MBR_ENT_NM_IX ON MBR_ENTRIES (UPPER_ENTRY_NAME); + +ALTER TABLE MBR_ITEMS + ADD CONSTRAINT PK_ITEMS + PRIMARY KEY (ITEM_ID_P1,ITEM_ID_P2,ITEM_VERSION); + +ALTER TABLE MBR_ITEM_CONTENTS + ADD CONSTRAINT PK_ITEM_CONTENTS + PRIMARY KEY (ITEM_ID_P1,ITEM_ID_P2,ITEM_VERSION); + +ALTER TABLE MBR_ENTRIES + ADD CONSTRAINT PK_ENTRIES + PRIMARY KEY (ENTRY_ID_P1,ENTRY_ID_P2); + +ALTER TABLE MBR_LABELS + ADD CONSTRAINT PK_LABELS + PRIMARY KEY (LABEL_ID_P1,LABEL_ID_P2); + +ALTER TABLE MBR_ITEM_LABELS + ADD CONSTRAINT PK_ITEM_LABELS + PRIMARY KEY (ITEM_ID_P1,ITEM_ID_P2,ITEM_VERSION,LABEL_ID_P1,LABEL_ID_P2); + +ALTER TABLE MBR_FOLDER_LABELS + ADD CONSTRAINT PK_FOLDER_LABELS + PRIMARY KEY (ENTRY_ID_P1,ENTRY_ID_P2,LABEL_ID_P1,LABEL_ID_P2); + +ALTER TABLE MBR_POLICIES + ADD CONSTRAINT PK_POLICIES + PRIMARY KEY (POLICY_NAME); +-- ========= STATEMENT 60 ============ +ALTER TABLE MBR_POL_PERMS + ADD CONSTRAINT PK_POL_PERMS + PRIMARY KEY (ENTRY_ID_P1,ENTRY_ID_P2,POLICY_NAME); + +ALTER TABLE MBR_POL_USERS + ADD CONSTRAINT PK_POL_USERS + PRIMARY KEY (POLICY_NAME,USER_NAME); +-- (generated from DtcBase/ObjectIndex) + + + +CREATE OR REPLACE VIEW MBR_READ_ENTRIES (ENTRY_ID_P1,ENTRY_ID_P2,USER_NAME) AS +SELECT MBR_POL_PERMS.ENTRY_ID_P1, MBR_POL_PERMS.ENTRY_ID_P2, + MBR_POL_USERS.USER_NAME +FROM MBR_POL_PERMS, MBR_POL_USERS , CS_SYSTEM_PROPS +where MBR_POL_PERMS.POLICY_NAME=MBR_POL_USERS.POLICY_NAME + AND (CS_SYSTEM_PROPS.PROPERTY_NAME='metamatrix.authorization.metabase.CheckingEnabled' + AND CS_SYSTEM_PROPS.PROPERTY_VALUE ='true' + AND MBR_POL_PERMS.READ_BIT='1') +UNION ALL +SELECT ENTRY_ID_P1, ENTRY_ID_P2, NULL +FROM MBR_ENTRIES ,CS_SYSTEM_PROPS +WHERE CS_SYSTEM_PROPS.PROPERTY_NAME='metamatrix.authorization.metabase.CheckingEnabled' + AND CS_SYSTEM_PROPS.PROPERTY_VALUE ='false' +; + + +CREATE INDEX MBR_POL_PERMS_IX1 ON MBR_POL_PERMS (POLICY_NAME, READ_BIT); + +CREATE INDEX LOGENTRIES_TMSTMP_IX ON LOGENTRIES (TIMESTAMP); + +CREATE TABLE DD_TXN_STATES +( + ID INTEGER primary key NOT NULL, + STATE VARCHAR(128) NOT NULL +); + +CREATE TABLE DD_TXN_LOG +( + ID BIGINT NOT NULL, + USER_NME VARCHAR(128), + BEGIN_TXN VARCHAR(50), + END_TXN VARCHAR(50), + ACTION VARCHAR(128), + TXN_STATE INTEGER +); + + +CREATE TABLE DD_SHREDQUEUE +( + QUEUE_ID NUMERIC(19) NOT NULL, + UUID1 NUMERIC(20) NOT NULL, + UUID2 NUMERIC(20) NOT NULL, + OBJECT_ID VARCHAR(44) NOT NULL, + NAME VARCHAR(128) NOT NULL, + VERSION VARCHAR(20), + MDL_PATH VARCHAR(2000), + CMD_ACTION NUMERIC(1) NOT NULL, + TXN_ID NUMERIC(19) , + SUB_BY_NME VARCHAR(100), + SUB_BY_DATE VARCHAR(50) +); + + +CREATE UNIQUE INDEX DDSQ_QUE_IX ON DD_SHREDQUEUE (QUEUE_ID) +; +CREATE UNIQUE INDEX DDSQ_TXN_IX ON DD_SHREDQUEUE (TXN_ID) +; +-- ========= STATEMENT 70 ============ +CREATE INDEX DDSQ_UUID_IX ON DD_SHREDQUEUE (OBJECT_ID) +; + +-- == new DTC start == +-- (generated from Models) + +CREATE TABLE MMR_MODELS +( + ID BIGINT NOT NULL, + NAME VARCHAR(256), + PATH VARCHAR(1024), + NAMESPACE VARCHAR(1024), + IS_METAMODEL SMALLINT, + VERSION VARCHAR(64), + IS_INCOMPLETE SMALLINT + -- SHRED_TIME DATETIME +); + +-- (generated from Resources) + +CREATE TABLE MMR_RESOURCES +( + MODEL_ID BIGINT NOT NULL, + CONTENT LONGTEXT NOT NULL +); + +-- (generated from Objects) + +CREATE TABLE MMR_OBJECTS +( + ID BIGINT NOT NULL, + MODEL_ID BIGINT NOT NULL, + NAME VARCHAR(256), + PATH VARCHAR(1024), + CLASS_NAME VARCHAR(256), + UUID VARCHAR(64), + NDX_PATH VARCHAR(256), + IS_UNRESOLVED SMALLINT +); + +-- (generated from ResolvedObjects) + +CREATE TABLE MMR_RESOLVED_OBJECTS +( + OBJ_ID BIGINT NOT NULL, + MODEL_ID BIGINT NOT NULL, + CLASS_ID BIGINT NOT NULL, + CONTAINER_ID BIGINT +); + +-- (generated from ReferenceFeatures) + +CREATE TABLE MMR_REF_FEATURES +( + MODEL_ID BIGINT NOT NULL, + OBJ_ID BIGINT NOT NULL, + NDX INT, + DATATYPE_ID BIGINT, + LOWER_BOUND INT, + UPPER_BOUND INT, + IS_CHANGEABLE SMALLINT, + IS_UNSETTABLE SMALLINT, + IS_CONTAINMENT SMALLINT, + OPPOSITE_ID BIGINT +); + +-- (generated from AttributeFeatures) + +CREATE TABLE MMR_ATTR_FEATURES +( + MODEL_ID BIGINT NOT NULL, + OBJ_ID BIGINT NOT NULL, + NDX INT, + DATATYPE_ID BIGINT, + LOWER_BOUND INT, + UPPER_BOUND INT, + IS_CHANGEABLE SMALLINT, + IS_UNSETTABLE SMALLINT +); + +-- (generated from References) + +CREATE TABLE MMR_REFS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + TO_ID BIGINT NOT NULL +); + +-- (generated from BooleanAttributes) + +CREATE TABLE MMR_BOOLEAN_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE SMALLINT NOT NULL +); + +-- (generated from ByteAttributes) + +CREATE TABLE MMR_BYTE_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE CHAR(1) NOT NULL +); +-- ========= STATEMENT 80 ============ +-- (generated from CharAttributes) + +CREATE TABLE MMR_CHAR_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE CHAR(1) +); + +-- (generated from ClobAttributes) + +CREATE TABLE MMR_CLOB_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE LONGTEXT +); + +-- (generated from DoubleAttributes) + +CREATE TABLE MMR_DOUBLE_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE DOUBLE NOT NULL +); + +-- (generated from EnumeratedAttributes) + +CREATE TABLE MMR_ENUM_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE INT NOT NULL +); + +-- (generated from FloatAttributes) + +CREATE TABLE MMR_FLOAT_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE FLOAT NOT NULL +); + +-- (generated from IntAttributes) + +CREATE TABLE MMR_INT_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE INT NOT NULL +); + +-- (generated from LongAttributes) + +CREATE TABLE MMR_LONG_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE BIGINT NOT NULL +); + + +-- (generated from ShortAttributes) + +CREATE TABLE MMR_SHORT_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE SMALLINT NOT NULL +); + +-- (generated from StringAttributes) + +CREATE TABLE MMR_STRING_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE VARCHAR(4000) +); + +-- Index length too long for MMR_MODELS(NAME,PATH) +CREATE INDEX MOD_PATH_NDX ON MMR_MODELS (NAME); +-- ========= STATEMENT 90 ============ +-- Specified key was too long; max key length is 3072 bytes +-- CREATE INDEX MOD_PATH2_NDX ON MMR_MODELS (PATH); + +-- CREATE INDEX MOD_NAMESPACE_NDX ON MMR_MODELS (NAMESPACE); + +CREATE INDEX OBJ_UUID_NDX ON MMR_OBJECTS (UUID); + +CREATE INDEX RES_OBJ_MODEL_NDX ON MMR_RESOLVED_OBJECTS (MODEL_ID); + +CREATE INDEX RES_OBJ_CLASS_NDX ON MMR_RESOLVED_OBJECTS (CLASS_ID); + +CREATE INDEX RF_DATATYPE_NDX ON MMR_REF_FEATURES (DATATYPE_ID); + +CREATE INDEX RF_MODEL_NDX ON MMR_REF_FEATURES (MODEL_ID); + +CREATE INDEX AF_DATATYPE_NDX ON MMR_ATTR_FEATURES (DATATYPE_ID); + +CREATE INDEX AF_MODEL_NDX ON MMR_ATTR_FEATURES (MODEL_ID); + +CREATE INDEX BOL_FEATURE_NDX ON MMR_BOOLEAN_ATTRS (FEATURE_ID); +-- ========= STATEMENT 100 ============ +CREATE INDEX BOL_MODEL_NDX ON MMR_BOOLEAN_ATTRS (MODEL_ID); + +CREATE INDEX BYT_FEATURE_NDX ON MMR_BYTE_ATTRS (FEATURE_ID); + +CREATE INDEX BYT_MODEL_NDX ON MMR_BYTE_ATTRS (MODEL_ID); + +CREATE INDEX CHR_FEATURE_NDX ON MMR_CHAR_ATTRS (FEATURE_ID); + +CREATE INDEX CHR_MODEL_NDX ON MMR_CHAR_ATTRS (MODEL_ID); + +CREATE INDEX CLOB_FEATURE_NDX ON MMR_CLOB_ATTRS (FEATURE_ID); + +CREATE INDEX CLOB_MODEL_NDX ON MMR_CLOB_ATTRS (MODEL_ID); + +CREATE INDEX DBL_FEATURE_NDX ON MMR_DOUBLE_ATTRS (FEATURE_ID); + +CREATE INDEX DBL_MODEL_NDX ON MMR_DOUBLE_ATTRS (MODEL_ID); + +CREATE INDEX ENUM_FEATURE_NDX ON MMR_ENUM_ATTRS (FEATURE_ID); +-- ========= STATEMENT 110 ============ +CREATE INDEX ENUM_MODEL_NDX ON MMR_ENUM_ATTRS (MODEL_ID); + +CREATE INDEX FLT_FEATURE_NDX ON MMR_FLOAT_ATTRS (FEATURE_ID); + +CREATE INDEX FLT_MODEL_NDX ON MMR_FLOAT_ATTRS (MODEL_ID); + +CREATE INDEX INT_FEATURE_NDX ON MMR_INT_ATTRS (FEATURE_ID); + +CREATE INDEX INT_MODEL_NDX ON MMR_INT_ATTRS (MODEL_ID); + +CREATE INDEX LNG_FEATURE_NDX ON MMR_LONG_ATTRS (FEATURE_ID); + +CREATE INDEX LNG_MODEL_NDX ON MMR_LONG_ATTRS (MODEL_ID); + +CREATE INDEX REF_FEATURE_NDX ON MMR_REFS (FEATURE_ID); + +CREATE INDEX REF_TO_NDX ON MMR_REFS (TO_ID); + +CREATE INDEX REF_MODEL_NDX ON MMR_REFS (MODEL_ID); +-- ========= STATEMENT 120 ============ +CREATE INDEX SHR_FEATURE_NDX ON MMR_SHORT_ATTRS (FEATURE_ID); + +CREATE INDEX SHR_MODEL_NDX ON MMR_SHORT_ATTRS (MODEL_ID); + +CREATE INDEX STR_FEATURE_NDX ON MMR_STRING_ATTRS (FEATURE_ID); + +CREATE INDEX STR_MODEL_NDX ON MMR_STRING_ATTRS (MODEL_ID); + +-- DATETIME length is incorrect +ALTER TABLE MMR_MODELS + ADD CONSTRAINT MOD_PK + PRIMARY KEY (ID); + +ALTER TABLE MMR_RESOURCES + ADD CONSTRAINT RSRC_PK + PRIMARY KEY (MODEL_ID); + +ALTER TABLE MMR_OBJECTS + ADD CONSTRAINT OBJ_PK + PRIMARY KEY (ID); + +ALTER TABLE MMR_RESOLVED_OBJECTS + ADD CONSTRAINT RES_OBJ_PK + PRIMARY KEY (OBJ_ID); + +ALTER TABLE MMR_REF_FEATURES + ADD CONSTRAINT RF_PK + PRIMARY KEY (OBJ_ID); + +ALTER TABLE MMR_ATTR_FEATURES + ADD CONSTRAINT AF_PK + PRIMARY KEY (OBJ_ID); +-- ========= STATEMENT 130 ============ +ALTER TABLE MMR_REFS + ADD CONSTRAINT REF_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_BOOLEAN_ATTRS + ADD CONSTRAINT BOL_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_BYTE_ATTRS + ADD CONSTRAINT BYT_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_CHAR_ATTRS + ADD CONSTRAINT CHR_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_CLOB_ATTRS + ADD CONSTRAINT CLOB_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_DOUBLE_ATTRS + ADD CONSTRAINT DBL_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_ENUM_ATTRS + ADD CONSTRAINT ENUM_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_FLOAT_ATTRS + ADD CONSTRAINT FLT_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_INT_ATTRS + ADD CONSTRAINT INT_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_LONG_ATTRS + ADD CONSTRAINT LNG_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); +-- ========= STATEMENT 140 ============ +ALTER TABLE MMR_SHORT_ATTRS + ADD CONSTRAINT SHR_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_STRING_ATTRS + ADD CONSTRAINT STR_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + + + -- View for obtaining the features by metaclass + -- (don't use parenthesis) + +CREATE OR REPLACE VIEW MMR_FEATURES AS + SELECT MMR_MODELS.NAMESPACE AS NAMESPACE, + PARENTS.NAME AS CLASS_NAME, + MMR_OBJECTS.NAME AS FEATURE_NAME, + MMR_ATTR_FEATURES.OBJ_ID AS FEATURE_ID, + 'Attribute' AS FEATURE_TYPE + FROM MMR_MODELS JOIN MMR_OBJECTS ON MMR_MODELS.ID=MMR_OBJECTS.MODEL_ID + JOIN MMR_ATTR_FEATURES ON MMR_OBJECTS.ID = MMR_ATTR_FEATURES.OBJ_ID + JOIN MMR_RESOLVED_OBJECTS ON MMR_OBJECTS.ID = MMR_RESOLVED_OBJECTS.OBJ_ID + JOIN MMR_OBJECTS PARENTS ON MMR_RESOLVED_OBJECTS.CONTAINER_ID = PARENTS.ID + UNION ALL + SELECT MMR_MODELS.NAMESPACE AS NAMESPACE, + PARENTS.NAME AS CLASS_NAME, + MMR_OBJECTS.NAME AS FEATURE_NAME, + MMR_REF_FEATURES.OBJ_ID AS FEATURE_ID, + 'Reference' AS FEATURE_TYPE + FROM MMR_MODELS JOIN MMR_OBJECTS ON MMR_MODELS.ID=MMR_OBJECTS.MODEL_ID + JOIN MMR_REF_FEATURES ON MMR_OBJECTS.ID = MMR_REF_FEATURES.OBJ_ID + JOIN MMR_RESOLVED_OBJECTS ON MMR_OBJECTS.ID = MMR_RESOLVED_OBJECTS.OBJ_ID + JOIN MMR_OBJECTS PARENTS ON MMR_RESOLVED_OBJECTS.CONTAINER_ID = PARENTS.ID + ; + + -- View for obtaining the feature values + -- (don't use parenthesis) + +CREATE OR REPLACE VIEW MMR_FEATURE_VALUES AS + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + VALUE AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_BOOLEAN_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + VALUE AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_BYTE_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + VALUE AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_CHAR_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + VALUE AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_DOUBLE_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + VALUE AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_FLOAT_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + VALUE AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_INT_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + VALUE AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_LONG_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + VALUE AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_SHORT_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + VALUE AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_STRING_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + VALUE AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_CLOB_ATTRS + UNION ALL + SELECT MMR_ENUM_ATTRS.OBJECT_ID, MMR_ENUM_ATTRS.MODEL_ID, MMR_ENUM_ATTRS.FEATURE_ID, MMR_ENUM_ATTRS.NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + MMR_OBJECTS.ID AS ENUM_ID, + MMR_REFS.NDX AS ENUM_VALUE, + MMR_OBJECTS.NAME AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_ENUM_ATTRS JOIN MMR_OBJECTS ON MMR_ENUM_ATTRS.VALUE = MMR_OBJECTS.ID + JOIN MMR_RESOLVED_OBJECTS ON MMR_OBJECTS.ID = MMR_RESOLVED_OBJECTS.OBJ_ID + JOIN MMR_REFS ON MMR_RESOLVED_OBJECTS.CONTAINER_ID = MMR_REFS.OBJECT_ID + AND MMR_RESOLVED_OBJECTS.OBJ_ID = MMR_REFS.TO_ID + UNION ALL + SELECT OBJECT_ID, MMR_REFS.MODEL_ID AS MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + MMR_OBJECTS.ID AS REF_OBJ_ID, + MMR_OBJECTS.NAME AS REF_OBJ_NAME + FROM MMR_REFS JOIN MMR_OBJECTS ON MMR_REFS.TO_ID = MMR_OBJECTS.ID; + +-- == new DTC end == + +-- Debezium output is null +-- INSERT INTO MMSCHEMAINFO_CA (SCRIPTNAME,SCRIPTEXECUTEDBY,SCRIPTREV, +-- RELEASEDATE, DATECREATED,DATEUPDATED, UPDATEID,METAMATRIXSERVERURL) +-- SELECT 'MM_CREATE.SQL',USER(),'Seneca.3117', '10/03/2008 12:01 AM',SYSDATE(),SYSDATE(),'',''; +-- ========= STATEMENT 145 ============ diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-ticketmonster-liquibase.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-ticketmonster-liquibase.ddl new file mode 100644 index 000000000..154075f67 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-ticketmonster-liquibase.ddl @@ -0,0 +1,84 @@ +CREATE TABLE `Appearance` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `event_id` bigint(20) DEFAULT NULL, + `event_name` varchar(255) DEFAULT NULL, + `venue_id` bigint(20) DEFAULT NULL, + `venue_name` varchar(255) DEFAULT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UKb2ol0eoqtadvfoxhsnqcajgqa` (`event_id`,`venue_id`) +) ENGINE=InnoDB AUTO_INCREMENT=7 DEFAULT CHARSET=latin1; + +CREATE TABLE `Booking` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `cancellationCode` varchar(255) NOT NULL, + `contactEmail` varchar(255) NOT NULL, + `createdOn` datetime(6) NOT NULL, + `performance_id` bigint(20) DEFAULT NULL, + `performance_name` varchar(255) DEFAULT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=8 DEFAULT CHARSET=latin1; + +CREATE TABLE `Section` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `description` varchar(255) NOT NULL, + `name` varchar(255) NOT NULL, + `numberOfRows` int(11) NOT NULL, + `rowCapacity` int(11) NOT NULL, + `venue_id` bigint(20) DEFAULT NULL, + `venue_name` varchar(255) DEFAULT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UKruosqireipse41rdsuvhqj050` (`name`,`venue_id`) +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=latin1; + +CREATE TABLE `SectionAllocation` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `allocated` longblob, + `occupiedCount` int(11) NOT NULL, + `performance_id` bigint(20) DEFAULT NULL, + `performance_name` varchar(255) DEFAULT NULL, + `version` bigint(20) NOT NULL, + `section_id` bigint(20) NOT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UK25wlm457x8dmc00we5uw7an3s` (`performance_id`,`section_id`), + KEY `FK60388cvbhb1xyrdhhe546t6dl` (`section_id`), + CONSTRAINT `FK60388cvbhb1xyrdhhe546t6dl` FOREIGN KEY (`section_id`) REFERENCES `Section` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION +) ENGINE=InnoDB AUTO_INCREMENT=57 DEFAULT CHARSET=latin1; + +CREATE TABLE `TicketCategory` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `description` varchar(255) NOT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UK_43455ipnchbn6r4bg8pviai3g` (`description`) +) ENGINE=InnoDB AUTO_INCREMENT=3 DEFAULT CHARSET=latin1; + +CREATE TABLE `Ticket` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `price` float NOT NULL, + `number` int(11) NOT NULL, + `rowNumber` int(11) NOT NULL, + `section_id` bigint(20) DEFAULT NULL, + `ticketCategory_id` bigint(20) NOT NULL, + `tickets_id` bigint(20) DEFAULT NULL, + PRIMARY KEY (`id`), + KEY `FK7xoel6i5b4nrphore8ns2jtld` (`section_id`), + KEY `FK88jejylfnpfqcslai19n4naqf` (`ticketCategory_id`), + KEY `FKolbt9u28gyshci6ek9ep0rl5d` (`tickets_id`), + CONSTRAINT `FK7xoel6i5b4nrphore8ns2jtld` FOREIGN KEY (`section_id`) REFERENCES `Section` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION, + CONSTRAINT `FK88jejylfnpfqcslai19n4naqf` FOREIGN KEY (`ticketCategory_id`) REFERENCES `TicketCategory` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION, + CONSTRAINT `FKolbt9u28gyshci6ek9ep0rl5d` FOREIGN KEY (`tickets_id`) REFERENCES `Booking` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION +) ENGINE=InnoDB AUTO_INCREMENT=13 DEFAULT CHARSET=latin1; + +CREATE TABLE `TicketPriceGuide` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `price` float NOT NULL, + `section_id` bigint(20) NOT NULL, + `show_id` bigint(20) NOT NULL, + `ticketCategory_id` bigint(20) NOT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UKro227lwq9ma9gy3ik6gl27xgm` (`section_id`,`show_id`,`ticketCategory_id`), + KEY `FK2nddwnrovke2wgpb8ffahqw` (`show_id`), + KEY `FK3d06sbv9l20tk2wa6yjsw9xdd` (`ticketCategory_id`), + CONSTRAINT `FK2nddwnrovke2wgpb8ffahqw` FOREIGN KEY (`show_id`) REFERENCES `Appearance` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION, + CONSTRAINT `FK3d06sbv9l20tk2wa6yjsw9xdd` FOREIGN KEY (`ticketCategory_id`) REFERENCES `TicketCategory` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION, + CONSTRAINT `FKaqmyqif55ipri4x65o8syt85k` FOREIGN KEY (`section_id`) REFERENCES `Section` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION +) ENGINE=InnoDB AUTO_INCREMENT=38 DEFAULT CHARSET=latin1; \ 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 48cf6071c..000000000 --- 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 index 6b60b39b9..d26e7d045 100644 --- a/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: numeric_column_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_751_numeric_column_test ( id INT AUTO_INCREMENT NOT NULL, rating1 NUMERIC, diff --git a/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql b/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql index 7d6da0e27..aae184d3f 100644 --- a/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql @@ -1,8 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- 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, diff --git a/tests/integration_tests/debezium/sql/debezium/regression_test.sql b/tests/integration_tests/debezium/sql/debezium/regression_test.sql index 5d396f4d4..2073c6fbb 100644 --- a/tests/integration_tests/debezium/sql/debezium/regression_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/regression_test.sql @@ -1,6 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- 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. @@ -42,6 +39,7 @@ CREATE TABLE dbz84_integer_types_table ( 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); +SET time_zone='UTC'; -- DBZ-85 handle fractional part of seconds CREATE TABLE dbz_85_fractest ( c1 DATE, @@ -86,7 +84,6 @@ 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, @@ -95,10 +92,13 @@ CREATE TABLE connector_test.customers ( 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"); +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"); +INSERT INTO connector_test.customers +VALUES (default,"George","Bailey","gbailey@foobar.com"); +INSERT INTO connector_test.customers +VALUES (default,"Edward","Walker","ed@walker.com"); +INSERT INTO connector_test.customers +VALUES (default,"Anne","Kretchmar","annek@noanswer.org"); -- DBZ-147 handle decimal value CREATE TABLE dbz_147_decimalvalues ( diff --git a/tests/integration_tests/debezium/sql/debezium/strategy_test.sql b/tests/integration_tests/debezium/sql/debezium/strategy_test.sql index a7ba9e50f..4575cfdb5 100644 --- a/tests/integration_tests/debezium/sql/debezium/strategy_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/strategy_test.sql @@ -1,3 +1,4 @@ +SET time_zone='UTC'; CREATE TABLE `dbz4180` ( id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, 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 index fc7b80aac..23a79a32a 100644 --- a/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- 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', diff --git a/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql b/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql index c80182cb5..84f55bd75 100644 --- a/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql @@ -1,3 +1,4 @@ +SET time_zone='UTC'; CREATE TABLE t_user_black_list ( `id` int(10) unsigned NOT NULL, `data` varchar(20), @@ -6,10 +7,11 @@ CREATE TABLE t_user_black_list ( 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; +-- Debezium `create_time` length is nil +-- 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'); +-- 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/topic_name_sanitization_test.sql b/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql index 878cb7cb2..2159c51b7 100644 --- a/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: topic-name.sanitization-it --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE `dbz_878_some|test@data` ( id INT, some_col VARCHAR(255), diff --git a/tests/integration_tests/debezium/sql/debezium/transaction_metadata_test.sql b/tests/integration_tests/debezium/sql/debezium/transaction_metadata_test.sql new file mode 100644 index 000000000..9ac761ad2 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/transaction_metadata_test.sql @@ -0,0 +1,26 @@ +-- 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 +); + +-- 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; + +-- 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; diff --git a/tests/integration_tests/debezium/sql/debezium/transactionpayload_test.sql b/tests/integration_tests/debezium/sql/debezium/transactionpayload_test.sql new file mode 100644 index 000000000..a1b4d3022 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/transactionpayload_test.sql @@ -0,0 +1,29 @@ +-- 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, + code BINARY(16) +); + +-- 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; + +-- 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; + +CREATE DATABASE IF NOT EXISTS transactionpayload_test; diff --git a/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql b/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql index 1ee023a63..638b9a5a0 100644 --- a/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql @@ -1,6 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- 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. 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 000000000..4766d56d9 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/year_test.sql @@ -0,0 +1,45 @@ +SET time_zone='UTC'; +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 f8fc3f564..ce5cf746d 100644 --- a/tests/integration_tests/debezium/sql/dml.sql +++ b/tests/integration_tests/debezium/sql/dml.sql @@ -11,9 +11,13 @@ INSERT INTO foo VALUES (3, 3); /* Update PK */ UPDATE foo SET PK = 5, COL = 5 WHERE COL = 3; +SELECT * FROM foo WHERE COL = 3; /* Update Multiple Rows */ UPDATE foo SET COL = 4; +SELECT * FROM foo WHERE PK = 1; +SELECT * FROM foo WHERE PK = 2; +SELECT * FROM foo WHERE PK = 3; /* Update Single Row */ UPDATE foo SET COL = 1 WHERE PK = 5; diff --git a/tests/integration_tests/debezium/src/main.go b/tests/integration_tests/debezium/src/main.go index d9abea1ca..452b32584 100644 --- a/tests/integration_tests/debezium/src/main.go +++ b/tests/integration_tests/debezium/src/main.go @@ -94,12 +94,7 @@ func main() { readerTiCDC = prepareKafkaConn(*topicTiCDC) defer readerTiCDC.Close() - dbMySQL = prepareDBConn(KindMySQL, *dbConnMySQL) - defer dbMySQL.MustClose() - dbTiDB = prepareDBConn(KindTiDB, *dbConnTiDB) - defer dbTiDB.MustClose() - - if !runAllTestCases(*testCaseDir) { + if !runAllTestCases(*testCaseDir, *dbConnMySQL, *dbConnTiDB) { os.Exit(1) } } diff --git a/tests/integration_tests/debezium/src/test_cases.go b/tests/integration_tests/debezium/src/test_cases.go index 8e3e05b1d..26ad99a8b 100644 --- a/tests/integration_tests/debezium/src/test_cases.go +++ b/tests/integration_tests/debezium/src/test_cases.go @@ -14,6 +14,7 @@ package main import ( + "bytes" "context" "encoding/json" "errors" @@ -29,10 +30,14 @@ import ( "github.com/google/go-cmp/cmp" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/format" + "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/segmentio/kafka-go" "go.uber.org/zap" ) +var timeOut = time.Second * 10 + var ( nFailed = 0 nPassed = 0 @@ -43,6 +48,18 @@ var ( msgValue = "value" ) +var defaultLength = map[string]float64{ + "INTEGER": 11, + "INTEGER UNSIGNED": 10, + "INT": 11, + "INT UNSIGNED": 10, + "TINYINT": 4, + "TINYINT UNSIGNED": 3, + "BIGINT": 20, + "BIT": 1, + "CHAR": 1, +} + func parseSQLText(data string) (res []ast.StmtNode, warns []error, err error) { p := parser.New() statements, warns, err := p.Parse(data, "utf8mb4", "utf8mb4_bin") @@ -70,25 +87,23 @@ func readAndParseSQLText(sqlFilePath string) []ast.StmtNode { return statements } -func runAllTestCases(dir string) bool { +func runAllTestCases(dir, dbConnMySQL, dbConnTiDB 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 + if strings.HasSuffix(info.Name(), ".sql") || strings.HasSuffix(info.Name(), ".ddl") { + files = append(files, path) } - 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) + runTestCase(path, dbConnMySQL, dbConnTiDB) } if nFailed > 0 { @@ -106,24 +121,32 @@ func runAllTestCases(dir string) bool { return nFailed == 0 } -func resetDB(db *DBHelper) { - db.MustExec("drop database if exists `" + *dbName + "`;") - db.MustExec("create database `" + *dbName + "`;") - db.MustExec("use `" + *dbName + "`;") +func resetDB() { + runSingleQuery("drop database if exists `"+*dbName+"`;", false) + runSingleQuery("create database `"+*dbName+"`;", false) + runSingleQuery("use `"+*dbName+"`;", false) } -func runTestCase(testCasePath string) bool { - resetDB(dbMySQL) - resetDB(dbTiDB) +func runTestCase(testCasePath, dbConnMySQL, dbConnTiDB string) { + dbMySQL = prepareDBConn(KindMySQL, dbConnMySQL) + defer dbMySQL.MustClose() + dbTiDB = prepareDBConn(KindTiDB, dbConnTiDB) + defer dbTiDB.MustClose() + + resetDB() statementKindsToWaitCDCRecord := map[string]bool{ - "Delete": true, - "Insert": true, - "Replace": true, - "Update": true, + "Delete": true, + "Insert": true, + "Replace": true, + "Update": true, + "CreateDatabase": true, + "DropDatabase": true, + "CreateTable": true, + "AlterTable": true, + "DropTable": true, } - hasError := false stmtAsts := readAndParseSQLText(testCasePath) for _, stmt := range stmtAsts { query := strings.TrimSpace(stmt.Text()) @@ -133,21 +156,18 @@ func runTestCase(testCasePath string) bool { 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, map[string]any, error) { + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() for { - ctx, cancel := context.WithTimeout(context.Background(), timeout) m, err := reader.FetchMessage(ctx) if err != nil { if errors.Is(err, context.DeadlineExceeded) { @@ -156,10 +176,9 @@ func fetchNextCDCRecord(reader *kafka.Reader, kind Kind, timeout time.Duration) return nil, nil, fmt.Errorf("Failed to read CDC record of %s: %w", kind, err) } - if err = reader.CommitMessages(context.Background(), m); err != nil { + if err = reader.CommitMessages(ctx, m); err != nil { return nil, nil, fmt.Errorf("Failed to commit CDC record of %s: %w", kind, err) } - cancel() if len(m.Value) == 0 { continue @@ -169,56 +188,76 @@ func fetchNextCDCRecord(reader *kafka.Reader, kind Kind, timeout time.Duration) var obj map[string]any err = json.Unmarshal(m.Key, &keyMap) if err != nil { - return nil, nil, fmt.Errorf("Failed to parse CDC record of %s (msg=%s): %w", kind, m.Key, err) + return nil, nil, fmt.Errorf("Failed to parse CDC record of %s (msg key=%s): %w", kind, m.Key, err) } err = json.Unmarshal(m.Value, &obj) if err != nil { - return nil, nil, fmt.Errorf("Failed to parse CDC record of %s (msg=%s): %w", kind, m.Value, err) + return nil, nil, fmt.Errorf("Failed to parse CDC record of %s (msg value=%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, nil, fmt.Errorf("Unexpected CDC record of %s: schema field not exist in %s", kind, m.Value) + payload, ok := obj["payload"].(map[string]any) + if !ok { + return nil, nil, fmt.Errorf("Unexpected CDC record of %s: payload field not exist in %s", kind, m.Value) + } + if kind == KindTiDB { + op, ok := payload["op"] + // Ignore Checkpoint events in the TiCDC's output + if ok && op == "m" { + continue } - if schema.(map[string]any)["name"] == "io.debezium.connector.mysql.SchemaChangeValue" { + // Only handle DDL received from partition-0 should be enough. + if !ok && m.Partition != 0 { continue } } + if ddl, ok := payload["ddl"]; ok { + payload["ddl"] = normalizeSQL(ddl.(string)) + } + // HACK + // In DDL events, some data types can't decode correctly, but the query is exact. + if kind == KindMySQL { + if tableChanges, ok := payload["tableChanges"]; ok { + if tables, ok := tableChanges.([]any); ok && len(tables) > 0 { + if table0, ok := tables[0].(map[string]any); ok { + if table, ok := table0["table"]; ok && table != nil { + if columns, ok := table.(map[string]any); ok { + for _, col := range columns["columns"].([]any) { + col := col.(map[string]any) + v := col["typeName"].(string) + switch v { + case "INT", "INT UNSIGNED", "INTEGER", "INTEGER UNSIGNED", "TINYINT", "TINYINT UNSIGNED", "BIGINT", + "BIT", "CHAR": + if col["length"] == defaultLength[v] { + col["length"] = nil + } + } + switch v { + case "INTEGER", "INTEGER UNSIGNED": + col["typeName"] = replaceString(col["typeName"], "INTEGER", "INT") + col["typeExpression"] = replaceString(col["typeExpression"], "INTEGER", "INT") + case "NUMERIC": + col["typeName"] = replaceString(col["typeName"], "NUMERIC", "DECIMAL") + col["typeExpression"] = replaceString(col["typeExpression"], "NUMERIC", "DECIMAL") + col["jdbcType"] = float64(3) + case "REAL": + col["typeName"] = replaceString(col["typeName"], "REAL", "FLOAT") + col["typeExpression"] = replaceString(col["typeExpression"], "REAL", "FLOAT") + col["jdbcType"] = float64(6) + } + } + } + } + } + } + } + } return keyMap, obj, nil } } -func replaceString(s any, key any, val any) string { - return strings.Replace(s.(string), key.(string), val.(string), 1) -} - -func fetchAllCDCRecords(reader *kafka.Reader, kind Kind) ([]map[string]any, []map[string]any) { - var records []map[string]any - var keyMaps []map[string]any - waitTimeout := 30 * time.Second - for { - keyMap, 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) - keyMaps = append(keyMaps, keyMap) - waitTimeout = time.Millisecond * 1000 - } - - return keyMaps, records +func replaceString(s any, old any, new any) string { + return strings.Replace(s.(string), old.(string), new.(string), 1) } var ignoredRecordPaths = map[string]bool{ @@ -228,10 +267,17 @@ var ignoredRecordPaths = map[string]bool{ var headingColor = color.New(color.FgHiWhite, color.Bold) -func printRecord(obj any) { - v, _ := json.MarshalIndent(obj, "", " ") - quick.Highlight(os.Stdout, string(v), "json", "terminal16m", "vs") - fmt.Println() +func normalizeSQL(sql string) string { + p := parser.New() + p.SetSQLMode(mysql.ModeRealAsFloat) // necessary + stmt, err := p.ParseOneStmt(sql, "", "") + buf := new(bytes.Buffer) + if err != nil { + panic(fmt.Sprintf("parse sql failed %s", err)) + } + restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, buf) + stmt.Restore(restoreCtx) + return buf.String() } func runSingleQuery(query string, waitCDCRows bool) bool { @@ -248,8 +294,22 @@ func runSingleQuery(query string, waitCDCRows bool) bool { }() wg.Wait() } - if !waitCDCRows { + wg := &sync.WaitGroup{} + wg.Add(2) + go func() { + if _, _, err := fetchNextCDCRecord(readerDebezium, KindMySQL, timeOut); err != nil { + logger.Error("fetch record failed", zap.Error(err)) + } + wg.Done() + }() + go func() { + if _, _, err := fetchNextCDCRecord(readerTiCDC, KindTiDB, timeOut); err != nil { + logger.Error("fetch record failed", zap.Error(err)) + } + wg.Done() + }() + wg.Wait() return true } @@ -263,48 +323,31 @@ func runSingleQuery(query string, waitCDCRows bool) bool { testCasePassed = false } - var keyMapsDebezium []map[string]any - var objsDebezium []map[string]any - var keyMapsTiCDC []map[string]any - var objsTiCDC []map[string]any + var keyMapsDebezium map[string]any + var objsDebezium map[string]any + var keyMapsTiCDC map[string]any + var objsTiCDC map[string]any { wg := &sync.WaitGroup{} wg.Add(2) go func() { - keyMapsDebezium, objsDebezium = fetchAllCDCRecords(readerDebezium, KindMySQL) + var err error + keyMapsDebezium, objsDebezium, err = fetchNextCDCRecord(readerDebezium, KindMySQL, timeOut) + if err != nil { + logger.Error("fetch record failed", zap.Error(err)) + } wg.Done() }() go func() { - keyMapsTiCDC, objsTiCDC = fetchAllCDCRecords(readerTiCDC, KindTiDB) + var err error + keyMapsTiCDC, objsTiCDC, err = fetchNextCDCRecord(readerTiCDC, KindTiDB, timeOut) + if err != nil { + logger.Error("fetch record failed", zap.Error(err)) + } wg.Done() }() wg.Wait() } - - diff(keyMapsDebezium, keyMapsTiCDC, onError, msgKey) - diff(objsDebezium, objsTiCDC, onError, msgValue) - - return testCasePassed -} - -func diff(recordsDebezium, recordsTiCDC []map[string]any, onError func(error), msgType string) { - if len(recordsDebezium) != len(recordsTiCDC) { - onError(fmt.Errorf( - "Mismatch CDC %s: Got %d record from Debezium and %d record from TiCDC", - msgType, - len(recordsDebezium), - len(recordsTiCDC))) - - headingColor.Print("\nDebezium output:\n\n") - for _, record := range recordsDebezium { - printRecord(record) - } - headingColor.Print("\nTiCDC output:\n\n") - for _, record := range recordsTiCDC { - printRecord(record) - } - return - } cmpOption := cmp.FilterPath( func(p cmp.Path) bool { path := p.GoString() @@ -314,15 +357,17 @@ func diff(recordsDebezium, recordsTiCDC []map[string]any, onError func(error), m cmp.Ignore(), ) - for i := 0; i < len(recordsDebezium); i++ { - recordDebezium := recordsDebezium[i] - recordTiCDC := recordsTiCDC[i] - if diff := cmp.Diff(recordDebezium, recordTiCDC, cmpOption); diff != "" { - onError(fmt.Errorf("Found mismatch CDC record (output record #%d)", i+1)) - headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") - quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") - fmt.Println() - continue - } + if diff := cmp.Diff(keyMapsDebezium, keyMapsTiCDC, cmpOption); diff != "" { + onError(fmt.Errorf("Found mismatch CDC record (msg type %s)", msgKey)) + headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") + quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") + fmt.Println() } + if diff := cmp.Diff(objsDebezium, objsTiCDC, cmpOption); diff != "" { + onError(fmt.Errorf("Found mismatch CDC record (msg type %s)", msgValue)) + headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") + quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") + fmt.Println() + } + return testCasePassed } diff --git a/tests/integration_tests/debezium_basic/conf/diff_config.toml b/tests/integration_tests/debezium_basic/conf/diff_config.toml new file mode 100644 index 000000000..d46eb33eb --- /dev/null +++ b/tests/integration_tests/debezium_basic/conf/diff_config.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] +output-dir = "/tmp/tidb_cdc_test/debezium_basic/output" + +source-instances = ["mysql1"] + +target-instance = "tidb0" + +target-check-tables = ["test.?*"] + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 4000 +user = "root" +password = "" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "" diff --git a/tests/integration_tests/debezium_basic/data/data.sql b/tests/integration_tests/debezium_basic/data/data.sql new file mode 100644 index 000000000..98154560a --- /dev/null +++ b/tests/integration_tests/debezium_basic/data/data.sql @@ -0,0 +1,330 @@ +drop database if exists test; +create database test; +use test; + +create table tp_int +( + id int auto_increment, + c_tinyint tinyint null, + c_smallint smallint null, + c_mediumint mediumint null, + c_int int null, + c_bigint bigint null, + constraint pk + primary key (id) +); + +insert into tp_int() +values (); + +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (1, 2, 3, 4, 5); + +-- insert max value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (127, 32767, 8388607, 2147483647, 9223372036854775807); + +-- insert min value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (-128, -32768, -8388608, -2147483648, -9223372036854775808); + +update tp_int set c_int = 0, c_tinyint = 0 where c_smallint = 2; +delete from tp_int where c_int = 0; + +-- unsigned int +create table tp_unsigned_int ( + id int auto_increment, + c_unsigned_tinyint tinyint unsigned null, + c_unsigned_smallint smallint unsigned null, + c_unsigned_mediumint mediumint unsigned null, + c_unsigned_int int unsigned null, + c_unsigned_bigint bigint unsigned null, + constraint pk + primary key (id) +); + +insert into tp_unsigned_int() +values (); + +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (1, 2, 3, 4, 5); + +-- insert max value +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (255, 65535, 16777215, 4294967295, 18446744073709551615); + +-- insert signed max value +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (127, 32767, 8388607, 2147483647, 9223372036854775807); + +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (128, 32768, 8388608, 2147483648, 9223372036854775808); + +update tp_unsigned_int set c_unsigned_int = 0, c_unsigned_tinyint = 0 where c_unsigned_smallint = 65535; +delete from tp_unsigned_int where c_unsigned_int = 0; + +-- real +create table tp_real +( + id int auto_increment, + c_float float null, + c_double double null, + c_decimal decimal null, + c_decimal_2 decimal(10, 4) null, + constraint pk + primary key (id) +); + +insert into tp_real() +values (); + +insert into tp_real(c_float, c_double, c_decimal, c_decimal_2) +values (2020.0202, 2020.0303, 2020.0404, 2021.1208); + +insert into tp_real(c_float, c_double, c_decimal, c_decimal_2) +values (-2.7182818284, -3.1415926, -8000, -179394.233); + +update tp_real set c_double = 2.333 where c_double = 2020.0303; + +-- unsigned real +create table tp_unsigned_real ( + id int auto_increment, + c_unsigned_float float unsigned null, + c_unsigned_double double unsigned null, + c_unsigned_decimal decimal unsigned null, + c_unsigned_decimal_2 decimal(10, 4) unsigned null, + constraint pk + primary key (id) +); + +insert into tp_unsigned_real() +values (); + +insert into tp_unsigned_real(c_unsigned_float, c_unsigned_double, c_unsigned_decimal, c_unsigned_decimal_2) +values (2020.0202, 2020.0303, 2020.0404, 2021.1208); + +update tp_unsigned_real set c_unsigned_double = 2020.0404 where c_unsigned_double = 2020.0303; + +-- time +create table tp_time +( + id int auto_increment, + c_date date null, + c_datetime datetime null, + c_timestamp timestamp null, + c_time time null, + c_year year null, + constraint pk + primary key (id) +); + +insert into tp_time() +values (); + +insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) +values ('2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020'); + +insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) +values ('2022-02-22', '2022-02-22 22:22:22', '2020-02-20 02:20:20', '02:20:20', '2021'); + +update tp_time set c_year = '2022' where c_year = '2020'; +update tp_time set c_date = '2022-02-22' where c_datetime = '2020-02-20 02:20:20'; + +-- text +create table tp_text +( + id int auto_increment, + c_tinytext tinytext null, + c_text text null, + c_mediumtext mediumtext null, + c_longtext longtext null, + constraint pk + primary key (id) +); + +insert into tp_text() +values (); + +insert into tp_text(c_tinytext, c_text, c_mediumtext, c_longtext) +values ('89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A'); + +insert into tp_text(c_tinytext, c_text, c_mediumtext, c_longtext) +values ('89504E470D0A1A0B', '89504E470D0A1A0B', '89504E470D0A1A0B', '89504E470D0A1A0B'); + +update tp_text set c_text = '89504E470D0A1A0B' where c_mediumtext = '89504E470D0A1A0A'; + +-- blob +create table tp_blob +( + id int auto_increment, + c_tinyblob tinyblob null, + c_blob blob null, + c_mediumblob mediumblob null, + c_longblob longblob null, + constraint pk + primary key (id) +); + +insert into tp_blob() +values (); + +insert into tp_blob(c_tinyblob, c_blob, c_mediumblob, c_longblob) +values (x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A'); + +insert into tp_blob(c_tinyblob, c_blob, c_mediumblob, c_longblob) +values (x'89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B'); + +update tp_blob set c_blob = x'89504E470D0A1A0B' where c_mediumblob = x'89504E470D0A1A0A'; + +-- char / binary +create table tp_char_binary +( + id int auto_increment, + c_char char(16) null, + c_varchar varchar(16) null, + c_binary binary(16) null, + c_varbinary varbinary(16) null, + constraint pk + primary key (id) +); + +insert into tp_char_binary() +values (); + +insert into tp_char_binary(c_char, c_varchar, c_binary, c_varbinary) +values ('89504E470D0A1A0A', '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A'); + +insert into tp_char_binary(c_char, c_varchar, c_binary, c_varbinary) +values ('89504E470D0A1A0B', '89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B'); + +update tp_char_binary set c_varchar = '89504E470D0A1A0B' where c_binary = x'89504E470D0A1A0A'; + +-- other +create table tp_other +( + id int auto_increment, + c_enum enum ('a','b','c') null, + c_set set ('a','b','c') null, + c_bit bit(64) null, + c_json json null, + constraint pk + primary key (id) +); + +insert into tp_other() +values (); + +insert into tp_other(c_enum, c_set, c_bit, c_json) +values ('a', 'a,b', b'1000001', '{ + "key1": "value1", + "key2": "value2" +}'); + +insert into tp_other(c_enum, c_set, c_bit, c_json) +values ('b', 'b,c', b'1000001', '{ + "key1": "value1", + "key2": "value2", + "key3": "123" +}'); + +update tp_other set c_enum = 'c' where c_set = 'b,c'; + +-- gbk dmls +CREATE TABLE cs_gbk ( + id INT, + name varchar(128) CHARACTER SET gbk, + country char(32) CHARACTER SET gbk, + city varchar(64), + description text CHARACTER SET gbk, + image tinyblob, + PRIMARY KEY (id) +) ENGINE = InnoDB CHARSET = utf8mb4; + +INSERT INTO cs_gbk +VALUES (1, '测试', "中国", "上海", "你好,世界" + , 0xC4E3BAC3CAC0BDE7); + +INSERT INTO cs_gbk +VALUES (2, '部署', "美国", "纽约", "世界,你好" + , 0xCAC0BDE7C4E3BAC3); + +UPDATE cs_gbk +SET name = '开发' +WHERE name = '测试'; + +DELETE FROM cs_gbk +WHERE name = '部署' + AND country = '美国' + AND city = '纽约' + AND description = '世界,你好'; + +-- ddls +CREATE TABLE test_ddl1 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +RENAME TABLE test_ddl1 TO test_ddl; + +ALTER TABLE test_ddl + ADD INDEX test_add_index (c1); + +DROP INDEX test_add_index ON test_ddl; + +ALTER TABLE test_ddl + ADD COLUMN c2 INT NOT NULL; + +TRUNCATE TABLE test_ddl; + +DROP TABLE test_ddl2; + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl3 ( + id INT, + 名称 varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +ALTER TABLE test_ddl3 + ADD COLUMN 城市 char(32); + +ALTER TABLE test_ddl3 + MODIFY COLUMN 城市 varchar(32); + +ALTER TABLE test_ddl3 + DROP COLUMN 城市; + +/* this is a DDL test for table */ +CREATE TABLE 表1 ( + id INT, + name varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +RENAME TABLE 表1 TO 表2; + +DROP TABLE 表2; + +create table finish_mark +( + id int PRIMARY KEY +); diff --git a/tests/integration_tests/debezium_basic/data/data_gbk.sql b/tests/integration_tests/debezium_basic/data/data_gbk.sql new file mode 100644 index 000000000..504d7b411 --- /dev/null +++ b/tests/integration_tests/debezium_basic/data/data_gbk.sql @@ -0,0 +1,101 @@ +-- This sql file is encoded in GBK by the 'iconv' command. +-- DO NOT EDIT. + +DROP DATABASE IF EXISTS test; +CREATE DATABASE test; +SET NAMES gbk; +USE test; + +-- gbk dmls +CREATE TABLE cs_gbk ( + id INT, + name varchar(128) CHARACTER SET gbk, + country char(32) CHARACTER SET gbk, + city varchar(64), + description text CHARACTER SET gbk, + image tinyblob, + PRIMARY KEY (id) +) ENGINE = InnoDB CHARSET = utf8mb4; + +INSERT INTO cs_gbk +VALUES (1, '', "й", "Ϻ", "," + , 0xC4E3BAC3CAC0BDE7); + +INSERT INTO cs_gbk +VALUES (2, '', "", "ŦԼ", "," + , 0xCAC0BDE7C4E3BAC3); + +UPDATE cs_gbk +SET name = '' +WHERE name = ''; + +DELETE FROM cs_gbk +WHERE name = '' + AND country = '' + AND city = 'ŦԼ' + AND description = ','; + +-- ddls +CREATE TABLE test_ddl1 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +RENAME TABLE test_ddl1 TO test_ddl; + +ALTER TABLE test_ddl + ADD INDEX test_add_index (c1); + +DROP INDEX test_add_index ON test_ddl; + +ALTER TABLE test_ddl + ADD COLUMN c2 INT NOT NULL; + +TRUNCATE TABLE test_ddl; + +DROP TABLE test_ddl2; + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl3 ( + id INT, + varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +ALTER TABLE test_ddl3 + ADD COLUMN char(32); + +ALTER TABLE test_ddl3 + MODIFY COLUMN varchar(32); + +ALTER TABLE test_ddl3 + DROP COLUMN ; + +/* this is a DDL test for table */ +CREATE TABLE 1 ( + id INT, + name varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +RENAME TABLE 1 TO 2; + +create table finish_mark +( + id int PRIMARY KEY +); diff --git a/tests/integration_tests/debezium_basic/run.sh b/tests/integration_tests/debezium_basic/run.sh new file mode 100644 index 000000000..734ecdc58 --- /dev/null +++ b/tests/integration_tests/debezium_basic/run.sh @@ -0,0 +1,53 @@ +#!/bin/bash + +set -e + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +# use kafka-consumer with debezium decoder to sync data from kafka to mysql +function run() { + if [ "$SINK_TYPE" != "kafka" ]; then + return + fi + + # clean up environment + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + # start tidb cluster + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + TOPIC_NAME="ticdc-debezium-basic-$RANDOM" + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=debezium&enable-tidb-extension=true" + + run_cdc_cli changefeed create --sink-uri="$SINK_URI" + sleep 5 # wait for changefeed to start + # determine the sink uri and run corresponding consumer + # currently only kafka and pulsar are supported + run_kafka_consumer $WORK_DIR $SINK_URI + + run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + # sync_diff can't check non-exist table, so we check expected tables are created in downstream first + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + run_sql_file $CUR/data/data_gbk.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/run_heavy_it_in_ci.sh b/tests/integration_tests/run_heavy_it_in_ci.sh index ad1a04856..8f58ca217 100755 --- a/tests/integration_tests/run_heavy_it_in_ci.sh +++ b/tests/integration_tests/run_heavy_it_in_ci.sh @@ -65,7 +65,7 @@ mysql_groups=( kafka_groups=( # G00 - '' + 'debezium' # G01 '' # G02 diff --git a/tests/integration_tests/run_light_it_in_ci.sh b/tests/integration_tests/run_light_it_in_ci.sh index 1505b646f..7af96f55f 100755 --- a/tests/integration_tests/run_light_it_in_ci.sh +++ b/tests/integration_tests/run_light_it_in_ci.sh @@ -94,7 +94,7 @@ kafka_groups=( # G11 'multi_topics' # G12 - '' + 'debezium_basic' # G13 'avro_basic' )