From ba1663cccdf0d63a863c753ac154ac512af4315c Mon Sep 17 00:00:00 2001 From: wk989898 Date: Mon, 12 Aug 2024 11:02:41 +0800 Subject: [PATCH 01/37] vector support --- cdc/entry/codec.go | 3 + cdc/entry/mounter.go | 3 + cdc/server/server.go | 2 +- cdc/sink/ddlsink/mysql/format_ddl.go | 55 +++ cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 4 + cdc/sink/dmlsink/txn/mysql/mysql.go | 10 + cdc/sink/dmlsink/txn/mysql/mysql_test.go | 26 + cdc/sink/util/vector.go | 43 ++ pkg/sink/codec/avro/avro.go | 58 ++- pkg/sink/codec/avro/avro_test.go | 547 +++++++++++++++++++++- pkg/sink/codec/canal/canal_entry.go | 2 + pkg/sink/codec/canal/canal_entry_test.go | 8 + pkg/sink/codec/craft/message_decoder.go | 4 + pkg/sink/codec/craft/message_encoder.go | 3 + pkg/sink/codec/csv/csv_message.go | 6 + pkg/sink/codec/csv/csv_message_test.go | 14 + pkg/sink/codec/internal/column.go | 2 + pkg/sink/codec/internal/java.go | 5 +- pkg/sink/codec/maxwell/maxwell_message.go | 11 + 19 files changed, 758 insertions(+), 48 deletions(-) create mode 100644 cdc/sink/ddlsink/mysql/format_ddl.go create mode 100644 cdc/sink/util/vector.go diff --git a/cdc/entry/codec.go b/cdc/entry/codec.go index 7224957d4d1..7828b7a1665 100644 --- a/cdc/entry/codec.go +++ b/cdc/entry/codec.go @@ -210,6 +210,9 @@ func unflatten(datum types.Datum, ft *types.FieldType, loc *time.Location) (type byteSize := (ft.GetFlen() + 7) >> 3 datum.SetUint64(0) datum.SetMysqlBit(types.NewBinaryLiteralFromUint(val, byteSize)) + case mysql.TypeTiDBVectorFloat32: + datum.SetVectorFloat32(types.ZeroVectorFloat32) + return datum, nil } return datum, nil } diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 0e511e47a78..7eec2a8bd9d 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -888,6 +888,9 @@ func formatColVal(datum types.Datum, col *timodel.ColumnInfo) ( } const sizeOfV = unsafe.Sizeof(v) return v, int(sizeOfV), warn, nil + case mysql.TypeTiDBVectorFloat32: + b := datum.GetVectorFloat32() + return b, b.Len(), "", nil default: // NOTICE: GetValue() may return some types that go sql not support, which will cause sink DML fail // Make specified convert upper if you need diff --git a/cdc/server/server.go b/cdc/server/server.go index 858b647ed55..205848779c7 100644 --- a/cdc/server/server.go +++ b/cdc/server/server.go @@ -215,7 +215,7 @@ func (s *server) setMemoryLimit() { } if conf.GcTunerMemoryThreshold > 0 { gctuner.EnableGOGCTuner.Store(true) - gctuner.Tuning(conf.GcTunerMemoryThreshold) + gctuner.Tuning(uint32(conf.GcTunerMemoryThreshold)) log.Info("enable gctuner, set memory limit", zap.Uint64("bytes", conf.GcTunerMemoryThreshold), zap.String("memory", humanize.IBytes(conf.GcTunerMemoryThreshold)), diff --git a/cdc/sink/ddlsink/mysql/format_ddl.go b/cdc/sink/ddlsink/mysql/format_ddl.go new file mode 100644 index 00000000000..b4390d96623 --- /dev/null +++ b/cdc/sink/ddlsink/mysql/format_ddl.go @@ -0,0 +1,55 @@ +// Copyright 2022 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 mysql + +import ( + "bytes" + + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/format" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/types" +) + +type fingerprintVisitor struct{} + +func (f *fingerprintVisitor) Enter(n ast.Node) (node ast.Node, skipChildren bool) { + if v, ok := n.(*ast.ColumnNameExpr); ok { + if v.Name.OrigColName() == "VECTOR" { + v.SetType(types.NewFieldType(mysql.TypeVarString)) + } + } + return n, false +} + +func (f *fingerprintVisitor) Leave(n ast.Node) (node ast.Node, ok bool) { + return n, true +} + +func formatQuery(sql string) string { + p := parser.New() + stmt, err := p.ParseOneStmt(sql, "", "") + if err != nil { + + } + stmt.Accept(&fingerprintVisitor{}) + + buf := new(bytes.Buffer) + restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, buf) + err = stmt.Restore(restoreCtx) + if nil != err { + } + return buf.String() +} diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 6d66a90e410..2e267525199 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -42,6 +42,8 @@ const ( // networkDriftDuration is used to construct a context timeout for database operations. networkDriftDuration = 5 * time.Second + + defaultSupportVectorVersion = "v8.3.0" ) // GetDBConnImpl is the implementation of pmysql.Factory. @@ -198,6 +200,8 @@ func (m *DDLSink) execDDL(pctx context.Context, ddl *model.DDLEvent) error { shouldSwitchDB := needSwitchDB(ddl) + m.formatQuery(ddl) + failpoint.Inject("MySQLSinkExecDDLDelay", func() { select { case <-ctx.Done(): diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index dea8debe570..16a0088324e 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -618,6 +618,16 @@ func (s *mysqlBackend) prepareDMLs() *preparedDMLs { callbacks = nil } + // decode vector + for i := 0; i < len(values); i++ { + value := values[i] + for j := 0; j < len(value); j++ { + switch value[j].(type) { + case types.VectorFloat32: + value[j] = value[j].(types.VectorFloat32).String() + } + } + } return &preparedDMLs{ startTs: startTs, sqls: sqls, diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index f0a9abb99a7..04c7da1b0a8 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tiflow/pkg/sink" pmysql "github.com/pingcap/tiflow/pkg/sink/mysql" "github.com/pingcap/tiflow/pkg/sqlmodel" + "github.com/pingcap/tiflow/pkg/util" "github.com/stretchr/testify/require" "go.uber.org/zap" "go.uber.org/zap/zaptest/observer" @@ -184,6 +185,31 @@ func TestPrepareDML(t *testing.T) { rowCount: 1, approximateSize: 63, }, + }, { + input: []*model.RowChangedEvent{ + { + StartTs: 418658114257813518, + CommitTs: 418658114257813519, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + Columns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeTiDBVectorFloat32, + Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), + }, { + Name: "a3", + Type: mysql.TypeTiDBVectorFloat32, + Value: util.Must(types.ParseVectorFloat32("[1.1,-2.0,3.33,-4.12,-5]")), + }}, + IndexColumns: [][]int{{}}, + }, + }, + expected: &preparedDMLs{ + startTs: []model.Ts{418658114257813518}, + sqls: []string{"REPLACE INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)"}, + values: [][]interface{}{{"[1,2,3,4,5]", "[1.1,-2.0,3.33,-4.12,-5]"}}, + rowCount: 1, + approximateSize: 64, + }, }, } diff --git a/cdc/sink/util/vector.go b/cdc/sink/util/vector.go new file mode 100644 index 00000000000..c5e5885f06e --- /dev/null +++ b/cdc/sink/util/vector.go @@ -0,0 +1,43 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "github.com/pingcap/tidb/table" + + "github.com/pingcap/tiflow/pkg/errors" +) + +// CheckVectorIndexForColumnModify checks Vector Index constraints for MODIFY COLUMN. +func CheckVectorIndexForColumnModify(oldCol *table.Column, newCol *table.Column) error { + if oldCol.VectorIndex == nil && newCol.VectorIndex == nil { + return nil + } + if oldCol.VectorIndex == nil && newCol.VectorIndex != nil { + return errors.Errorf("currently HNSW index can be only defined when creating the table") + } + if oldCol.VectorIndex != nil && newCol.VectorIndex == nil { + return errors.Errorf("currently HNSW index can not be removed") + } + if oldCol.FieldType.GetFlen() != newCol.FieldType.GetFlen() { + return errors.New("cannot modify vector column's dimention when HNSW index is defined") + } + if oldCol.FieldType.GetType() != newCol.FieldType.GetType() { + return errors.New("cannot modify column data type when HNSW index is defined") + } + if *(oldCol.VectorIndex) != *(newCol.VectorIndex) { + return errors.New("currently HNSW index cannot be modified") + } + return nil +} diff --git a/pkg/sink/codec/avro/avro.go b/pkg/sink/codec/avro/avro.go index 10222296a06..9847464bcb4 100644 --- a/pkg/sink/codec/avro/avro.go +++ b/pkg/sink/codec/avro/avro.go @@ -359,30 +359,31 @@ const ( ) var type2TiDBType = map[byte]string{ - mysql.TypeTiny: "INT", - mysql.TypeShort: "INT", - mysql.TypeInt24: "INT", - mysql.TypeLong: "INT", - mysql.TypeLonglong: "BIGINT", - mysql.TypeFloat: "FLOAT", - mysql.TypeDouble: "DOUBLE", - mysql.TypeBit: "BIT", - mysql.TypeNewDecimal: "DECIMAL", - mysql.TypeTinyBlob: "TEXT", - mysql.TypeMediumBlob: "TEXT", - mysql.TypeBlob: "TEXT", - mysql.TypeLongBlob: "TEXT", - mysql.TypeVarchar: "TEXT", - mysql.TypeVarString: "TEXT", - mysql.TypeString: "TEXT", - mysql.TypeEnum: "ENUM", - mysql.TypeSet: "SET", - mysql.TypeJSON: "JSON", - mysql.TypeDate: "DATE", - mysql.TypeDatetime: "DATETIME", - mysql.TypeTimestamp: "TIMESTAMP", - mysql.TypeDuration: "TIME", - mysql.TypeYear: "YEAR", + mysql.TypeTiny: "INT", + mysql.TypeShort: "INT", + mysql.TypeInt24: "INT", + mysql.TypeLong: "INT", + mysql.TypeLonglong: "BIGINT", + mysql.TypeFloat: "FLOAT", + mysql.TypeDouble: "DOUBLE", + mysql.TypeBit: "BIT", + mysql.TypeNewDecimal: "DECIMAL", + mysql.TypeTinyBlob: "TEXT", + mysql.TypeMediumBlob: "TEXT", + mysql.TypeBlob: "TEXT", + mysql.TypeLongBlob: "TEXT", + mysql.TypeVarchar: "TEXT", + mysql.TypeVarString: "TEXT", + mysql.TypeString: "TEXT", + mysql.TypeEnum: "ENUM", + mysql.TypeSet: "SET", + mysql.TypeJSON: "JSON", + mysql.TypeDate: "DATE", + mysql.TypeDatetime: "DATETIME", + mysql.TypeTimestamp: "TIMESTAMP", + mysql.TypeDuration: "TIME", + mysql.TypeYear: "YEAR", + mysql.TypeTiDBVectorFloat32: "TiDBVECTORFloat32", } func getTiDBTypeFromColumn(col *model.Column) string { @@ -812,6 +813,12 @@ func (a *BatchEncoder) columnToAvroSchema( Type: "int", Parameters: map[string]string{tidbType: tt}, }, nil + case mysql.TypeTiDBVectorFloat32: + return avroSchema{ + Type: "string", + // Type: "array", + Parameters: map[string]string{tidbType: tt}, + }, nil default: log.Error("unknown mysql type", zap.Any("mysqlType", col.Type)) return nil, cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") @@ -971,6 +978,9 @@ func (a *BatchEncoder) columnToAvroData( return int32(n), "int", nil } return int32(col.Value.(int64)), "int", nil + case mysql.TypeTiDBVectorFloat32: + vec := col.Value.(types.VectorFloat32) + return vec.String(), "string", nil default: log.Error("unknown mysql type", zap.Any("value", col.Value), zap.Any("mysqlType", col.Type)) return nil, "", cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") diff --git a/pkg/sink/codec/avro/avro_test.go b/pkg/sink/codec/avro/avro_test.go index 4be0ee03091..fcf98e9cfc7 100644 --- a/pkg/sink/codec/avro/avro_test.go +++ b/pkg/sink/codec/avro/avro_test.go @@ -25,36 +25,539 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" - "github.com/pingcap/tiflow/pkg/sink/codec/utils" - "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/require" ) -func TestDMLEventE2E(t *testing.T) { - codecConfig := common.NewConfig(config.ProtocolAvro) - codecConfig.EnableTiDBExtension = true - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() +func setupEncoderAndSchemaRegistry( + enableTiDBExtension bool, + decimalHandlingMode string, + bigintUnsignedHandlingMode string, +) (*BatchEncoder, error) { + startHTTPInterceptForTestingRegistry() + + keyManager, err := NewAvroSchemaManager( + context.Background(), + nil, + "http://127.0.0.1:8081", + "-key", + ) + if err != nil { + return nil, err + } - _, event, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) - colInfos := event.TableInfo.GetColInfosForRowChangedEvent() + valueManager, err := NewAvroSchemaManager( + context.Background(), + nil, + "http://127.0.0.1:8081", + "-value", + ) + if err != nil { + return nil, err + } - rand.New(rand.NewSource(time.Now().Unix())).Shuffle(len(event.Columns), func(i, j int) { - event.Columns[i], event.Columns[j] = event.Columns[j], event.Columns[i] - colInfos[i], colInfos[j] = colInfos[j], colInfos[i] - }) + return &BatchEncoder{ + namespace: model.DefaultNamespace, + valueSchemaManager: valueManager, + keySchemaManager: keyManager, + result: make([]*common.Message, 0, 1), + enableTiDBExtension: enableTiDBExtension, + decimalHandlingMode: decimalHandlingMode, + bigintUnsignedHandlingMode: bigintUnsignedHandlingMode, + }, nil +} - for _, decimalHandling := range []string{"precise", "string"} { - for _, unsignedBigintHandling := range []string{"long", "string"} { - codecConfig.AvroDecimalHandlingMode = decimalHandling - codecConfig.AvroBigintUnsignedHandlingMode = unsignedBigintHandling +func teardownEncoderAndSchemaRegistry() { + stopHTTPInterceptForTestingRegistry() +} - encoder, err := SetupEncoderAndSchemaRegistry4Testing(ctx, codecConfig) - require.NoError(t, err) - require.NotNil(t, encoder) +type avroTestColumnTuple struct { + col model.Column + colInfo rowcodec.ColInfo + expectedSchema interface{} + expectedData interface{} + expectedType string +} + +var avroTestColumns = []*avroTestColumnTuple{ + { + model.Column{Name: "tiny", Value: int64(1), Type: mysql.TypeTiny}, + rowcodec.ColInfo{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiny), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, + int32(1), "int", + }, + { + model.Column{Name: "short", Value: int64(1), Type: mysql.TypeShort}, + rowcodec.ColInfo{ + ID: 2, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeShort), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, + int32(1), "int", + }, + { + model.Column{Name: "int24", Value: int64(1), Type: mysql.TypeInt24}, + rowcodec.ColInfo{ + ID: 3, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeInt24), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, + int32(1), "int", + }, + { + model.Column{Name: "long", Value: int64(1), Type: mysql.TypeLong}, + rowcodec.ColInfo{ + ID: 4, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLong), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, + int32(1), "int", + }, + { + model.Column{Name: "longlong", Value: int64(1), Type: mysql.TypeLonglong}, + rowcodec.ColInfo{ + ID: 5, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLonglong), + }, + avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "BIGINT"}}, + int64(1), "long", + }, + { + model.Column{ + Name: "tinyunsigned", + Value: uint64(1), + Type: mysql.TypeTiny, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 6, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeTiny)), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, + int32(1), "int", + }, + { + model.Column{ + Name: "shortunsigned", + Value: uint64(1), + Type: mysql.TypeShort, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 7, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeShort)), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, + int32(1), "int", + }, + { + model.Column{ + Name: "int24unsigned", + Value: uint64(1), + Type: mysql.TypeInt24, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 8, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeInt24)), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, + int32(1), "int", + }, + { + model.Column{ + Name: "longunsigned", + Value: uint64(1), + Type: mysql.TypeLong, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 9, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeLong)), + }, + avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, + int64(1), "long", + }, + { + model.Column{ + Name: "longlongunsigned", + Value: uint64(1), + Type: mysql.TypeLonglong, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 10, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeLonglong)), + }, + avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "BIGINT UNSIGNED"}}, + int64(1), "long", + }, + { + model.Column{Name: "float", Value: float32(3.14), Type: mysql.TypeFloat}, + rowcodec.ColInfo{ + ID: 11, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeFloat), + }, + avroSchema{Type: "float", Parameters: map[string]string{"tidb_type": "FLOAT"}}, + float32(3.14), "float", + }, + { + model.Column{Name: "double", Value: float64(3.14), Type: mysql.TypeDouble}, + rowcodec.ColInfo{ + ID: 12, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDouble), + }, + avroSchema{Type: "double", Parameters: map[string]string{"tidb_type": "DOUBLE"}}, + float64(3.14), "double", + }, + { + model.Column{Name: "bit", Value: uint64(683), Type: mysql.TypeBit}, + rowcodec.ColInfo{ + ID: 13, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBit), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BIT", "length": "1"}}, + []byte("\x02\xab"), "bytes", + }, + { + model.Column{Name: "decimal", Value: "129012.1230000", Type: mysql.TypeNewDecimal}, + rowcodec.ColInfo{ + ID: 14, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeNewDecimal), + }, + avroLogicalTypeSchema{ + avroSchema: avroSchema{ + Type: "bytes", + Parameters: map[string]string{"tidb_type": "DECIMAL"}, + }, + LogicalType: "decimal", + Precision: 10, + Scale: 0, + }, + big.NewRat(129012123, 1000), "bytes.decimal", + }, + { + model.Column{Name: "tinytext", Value: []byte("hello world"), Type: mysql.TypeTinyBlob}, + rowcodec.ColInfo{ + ID: 15, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBlob), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "mediumtext", Value: []byte("hello world"), Type: mysql.TypeMediumBlob}, + rowcodec.ColInfo{ + ID: 16, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeMediumBlob), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "text", Value: []byte("hello world"), Type: mysql.TypeBlob}, + rowcodec.ColInfo{ + ID: 17, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBlob), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "longtext", Value: []byte("hello world"), Type: mysql.TypeLongBlob}, + rowcodec.ColInfo{ + ID: 18, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLongBlob), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "varchar", Value: []byte("hello world"), Type: mysql.TypeVarchar}, + rowcodec.ColInfo{ + ID: 19, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarchar), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "varstring", Value: []byte("hello world"), Type: mysql.TypeVarString}, + rowcodec.ColInfo{ + ID: 20, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarString), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "string", Value: []byte("hello world"), Type: mysql.TypeString}, + rowcodec.ColInfo{ + ID: 21, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeString), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{ + Name: "tinyblob", + Value: []byte("hello world"), + Type: mysql.TypeTinyBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 22, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "mediumblob", + Value: []byte("hello world"), + Type: mysql.TypeMediumBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 23, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "blob", + Value: []byte("hello world"), + Type: mysql.TypeBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 24, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "longblob", + Value: []byte("hello world"), + Type: mysql.TypeLongBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 25, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "varbinary", + Value: []byte("hello world"), + Type: mysql.TypeVarchar, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 26, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "varbinary1", + Value: []byte("hello world"), + Type: mysql.TypeVarString, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 27, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "binary", + Value: []byte("hello world"), + Type: mysql.TypeString, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 28, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeString)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{Name: "enum", Value: uint64(1), Type: mysql.TypeEnum}, + rowcodec.ColInfo{ + ID: 29, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetElems(types.NewFieldType(mysql.TypeEnum), []string{"a,", "b"}), + }, + avroSchema{ + Type: "string", + Parameters: map[string]string{"tidb_type": "ENUM", "allowed": "a\\,,b"}, + }, + "a,", "string", + }, + { + model.Column{Name: "set", Value: uint64(1), Type: mysql.TypeSet}, + rowcodec.ColInfo{ + ID: 30, + IsPKHandle: false, + VirtualGenCol: false, + Ft: common.SetElems(types.NewFieldType(mysql.TypeSet), []string{"a,", "b"}), + }, + avroSchema{ + Type: "string", + Parameters: map[string]string{"tidb_type": "SET", "allowed": "a\\,,b"}, + }, + "a,", "string", + }, + { + model.Column{Name: "json", Value: `{"key": "value"}`, Type: mysql.TypeJSON}, + rowcodec.ColInfo{ + ID: 31, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeJSON), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "JSON"}}, + `{"key": "value"}`, "string", + }, + { + model.Column{Name: "date", Value: "2000-01-01", Type: mysql.TypeDate}, + rowcodec.ColInfo{ + ID: 32, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDate), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "DATE"}}, + "2000-01-01", "string", + }, + { + model.Column{Name: "datetime", Value: "2015-12-20 23:58:58", Type: mysql.TypeDatetime}, + rowcodec.ColInfo{ + ID: 33, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDatetime), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "DATETIME"}}, + "2015-12-20 23:58:58", "string", + }, + { + model.Column{Name: "timestamp", Value: "1973-12-30 15:30:00", Type: mysql.TypeTimestamp}, + rowcodec.ColInfo{ + ID: 34, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTimestamp), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TIMESTAMP"}}, + "1973-12-30 15:30:00", "string", + }, + { + model.Column{Name: "time", Value: "23:59:59", Type: mysql.TypeDuration}, + rowcodec.ColInfo{ + ID: 35, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDuration), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TIME"}}, + "23:59:59", "string", + }, + { + model.Column{Name: "year", Value: int64(1970), Type: mysql.TypeYear}, + rowcodec.ColInfo{ + ID: 36, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeYear), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "YEAR"}}, + int32(1970), "int", + }, +} - topic := "avro-test-topic" - err = encoder.AppendRowChangedEvent(ctx, topic, event, func() {}) +func TestColumnToAvroSchema(t *testing.T) { + for _, v := range avroTestColumns { + schema, err := columnToAvroSchema(&v.col, v.colInfo.Ft, "precise", "long") + require.NoError(t, err) + require.Equal(t, v.expectedSchema, schema) + if v.col.Name == "decimal" { + schema, err := columnToAvroSchema(&v.col, v.colInfo.Ft, "string", "long") require.NoError(t, err) messages := encoder.Build() diff --git a/pkg/sink/codec/canal/canal_entry.go b/pkg/sink/codec/canal/canal_entry.go index e4724a525c1..9bf77f73a6b 100644 --- a/pkg/sink/codec/canal/canal_entry.go +++ b/pkg/sink/codec/canal/canal_entry.go @@ -109,6 +109,8 @@ func (b *canalEntryBuilder) formatValue(value interface{}, isBinary bool) (resul } else { result = string(v) } + case types.VectorFloat32: + result = value.(types.VectorFloat32).String() default: result = fmt.Sprintf("%v", v) } diff --git a/pkg/sink/codec/canal/canal_entry_test.go b/pkg/sink/codec/canal/canal_entry_test.go index 385c595dfcf..4735fd0cfef 100644 --- a/pkg/sink/codec/canal/canal_entry_test.go +++ b/pkg/sink/codec/canal/canal_entry_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/sink/codec/internal" + "github.com/pingcap/tiflow/pkg/util" canal "github.com/pingcap/tiflow/proto/canal" "github.com/stretchr/testify/require" "golang.org/x/text/encoding/charmap" @@ -97,6 +98,13 @@ func TestInsert(t *testing.T) { require.NoError(t, err) require.Equal(t, "测试blob", s) require.Equal(t, "blob", col.GetMysqlType()) + case "vectorfloat32": + require.Equal(t, int32(internal.JavaSQLTypeVARCHAR), col.GetSqlType()) + require.False(t, col.GetIsKey()) + require.False(t, col.GetIsNull()) + require.NoError(t, err) + require.Equal(t, "[1,2,3,4,5]", col.GetValue()) + require.Equal(t, "text", col.GetMysqlType()) } } } diff --git a/pkg/sink/codec/craft/message_decoder.go b/pkg/sink/codec/craft/message_decoder.go index 81d7bd88509..4a6fbec8f87 100644 --- a/pkg/sink/codec/craft/message_decoder.go +++ b/pkg/sink/codec/craft/message_decoder.go @@ -353,6 +353,10 @@ func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ fallthrough case mysql.TypeGeometry: return nil, nil + case mysql.TypeTiDBVectorFloat32: + if val, err := types.ParseVectorFloat32(string(bits)); err != nil { + return val, nil + } } return nil, nil } diff --git a/pkg/sink/codec/craft/message_encoder.go b/pkg/sink/codec/craft/message_encoder.go index 92fd3b3d5e4..7f219092f19 100644 --- a/pkg/sink/codec/craft/message_encoder.go +++ b/pkg/sink/codec/craft/message_encoder.go @@ -220,6 +220,9 @@ func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagTyp fallthrough case mysql.TypeGeometry: return nil + case mysql.TypeTiDBVectorFloat32: + vec := value.(types.VectorFloat32) + return []byte(vec.String()) } return nil } diff --git a/pkg/sink/codec/csv/csv_message.go b/pkg/sink/codec/csv/csv_message.go index 32486b2fe82..36bfae82219 100644 --- a/pkg/sink/codec/csv/csv_message.go +++ b/pkg/sink/codec/csv/csv_message.go @@ -311,6 +311,9 @@ func fromCsvValToColValue(csvConfig *common.Config, csvVal any, ft types.FieldTy case mysql.TypeBit: val, err := strconv.ParseUint(str, 10, 64) return val, err + case mysql.TypeTiDBVectorFloat32: + vec, err := types.ParseVectorFloat32(str) + return vec.String(), err default: return str, nil } @@ -362,6 +365,9 @@ func fromColValToCsvVal(csvConfig *common.Config, col *model.Column, ft *types.F return nil, cerror.WrapError(cerror.ErrCSVEncodeFailed, err) } return setVar.Name, nil + case mysql.TypeTiDBVectorFloat32: + vec := col.Value.(types.VectorFloat32) + return vec.String(), nil default: return col.Value, nil } diff --git a/pkg/sink/codec/csv/csv_message_test.go b/pkg/sink/codec/csv/csv_message_test.go index 57fe5f96010..3ce686bcb33 100644 --- a/pkg/sink/codec/csv/csv_message_test.go +++ b/pkg/sink/codec/csv/csv_message_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/util" "github.com/stretchr/testify/require" ) @@ -629,6 +630,19 @@ var csvTestColumnsGroup = [][]*csvTestColumnTuple{ config.BinaryEncodingBase64, }, }, + { + { + model.Column{Name: "vectorfloat32", Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), Type: mysql.TypeTiDBVectorFloat32}, + rowcodec.ColInfo{ + ID: 37, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiDBVectorFloat32), + }, + "[1,2,3,4,5]", + config.BinaryEncodingBase64, + }, + }, } func setBinChsClnFlag(ft *types.FieldType) *types.FieldType { diff --git a/pkg/sink/codec/internal/column.go b/pkg/sink/codec/internal/column.go index 5aefe9d9a16..916d3dfa89d 100644 --- a/pkg/sink/codec/internal/column.go +++ b/pkg/sink/codec/internal/column.go @@ -61,6 +61,8 @@ func (c *Column) FromRowChangeColumn(col *model.Column) { str = str[1 : len(str)-1] } c.Value = str + case mysql.TypeTiDBVectorFloat32: + c.Value = col.Value.(types.VectorFloat32).String() default: c.Value = col.Value } diff --git a/pkg/sink/codec/internal/java.go b/pkg/sink/codec/internal/java.go index 38ec2e33f6c..426f537c04a 100644 --- a/pkg/sink/codec/internal/java.go +++ b/pkg/sink/codec/internal/java.go @@ -48,7 +48,7 @@ const ( // JavaSQLTypeJAVA_OBJECT JavaSQLType = 2000 // JavaSQLTypeDISTINCT JavaSQLType = 2001 // JavaSQLTypeSTRUCT JavaSQLType = 2002 - // JavaSQLTypeARRAY JavaSQLType = 2003 + // JavaSQLTypeARRAY JavaSQLType = 2003 // JavaSQLTypeREF JavaSQLType = 2006 // JavaSQLTypeDATALINK JavaSQLType = 70 // JavaSQLTypeBOOLEAN JavaSQLType = 16 @@ -143,6 +143,9 @@ func MySQLType2JavaType(mysqlType byte, isBinary bool) JavaSQLType { case mysql.TypeJSON: return JavaSQLTypeVARCHAR + case mysql.TypeTiDBVectorFloat32: + return JavaSQLTypeVARCHAR + default: return JavaSQLTypeVARCHAR } diff --git a/pkg/sink/codec/maxwell/maxwell_message.go b/pkg/sink/codec/maxwell/maxwell_message.go index 9d0de05db53..c0bb8942457 100644 --- a/pkg/sink/codec/maxwell/maxwell_message.go +++ b/pkg/sink/codec/maxwell/maxwell_message.go @@ -84,6 +84,8 @@ func rowChangeToMaxwellMsg(e *model.RowChangedEvent, onlyHandleKeyColumns bool) } else { value.Old[colName] = string(v.Value.([]byte)) } + case mysql.TypeTiDBVectorFloat32: + value.Old[v.Name] = v.Value.(types.VectorFloat32).String() default: value.Old[colName] = v.Value } @@ -102,6 +104,8 @@ func rowChangeToMaxwellMsg(e *model.RowChangedEvent, onlyHandleKeyColumns bool) } else { value.Data[colName] = string(v.Value.([]byte)) } + case mysql.TypeTiDBVectorFloat32: + value.Data[v.Name] = v.Value.(types.VectorFloat32).String() default: value.Data[colName] = v.Value } @@ -129,6 +133,11 @@ func rowChangeToMaxwellMsg(e *model.RowChangedEvent, onlyHandleKeyColumns bool) value.Old[colName] = string(v.Value.([]byte)) } } + case mysql.TypeTiDBVectorFloat32: + val := v.Value.(types.VectorFloat32).String() + if value.Old[v.Name] != val { + value.Old[v.Name] = val + } default: if value.Data[colName] != v.Value { value.Old[colName] = v.Value @@ -282,6 +291,8 @@ func columnToMaxwellType(columnType byte) (string, error) { return "float", nil case mysql.TypeNewDecimal: return "decimal", nil + case mysql.TypeTiDBVectorFloat32: + return "string", nil default: return "", cerror.ErrMaxwellInvalidData.GenWithStack("unsupported column type - %v", columnType) } From 20d340bf5890f41b7a5e4a987e46c0295b1acee6 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Mon, 12 Aug 2024 11:31:25 +0800 Subject: [PATCH 02/37] update --- cdc/sink/ddlsink/mysql/format_ddl.go | 10 +- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 17 + cdc/sink/dmlsink/txn/mysql/mysql_test.go | 41 +- cdc/sink/util/vector.go | 2 +- pkg/sink/codec/avro/avro_test.go | 547 +--------------------- pkg/sink/codec/canal/canal_entry_test.go | 6 +- pkg/sink/codec/maxwell/maxwell_message.go | 6 +- 7 files changed, 78 insertions(+), 551 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/format_ddl.go b/cdc/sink/ddlsink/mysql/format_ddl.go index b4390d96623..79c1249c49b 100644 --- a/cdc/sink/ddlsink/mysql/format_ddl.go +++ b/cdc/sink/ddlsink/mysql/format_ddl.go @@ -16,11 +16,11 @@ package mysql import ( "bytes" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/ast" - "github.com/pingcap/tidb/parser/format" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/types" + "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/pingcap/tidb/pkg/types" ) type fingerprintVisitor struct{} diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 2e267525199..1d3822a0257 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -20,9 +20,12 @@ import ( "net/url" "time" + "github.com/coreos/go-semver/semver" lru "github.com/hashicorp/golang-lru" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/pingcap/tidb/br/pkg/version" + "github.com/pingcap/tidb/dumpling/export" timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/ddlsink" @@ -296,3 +299,17 @@ func (m *DDLSink) Close() { } } } + +func (m *DDLSink) formatQuery(ddl *model.DDLEvent) { + versionInfo, err := export.SelectVersion(m.db) + if err != nil { + log.Error("fail to get version info", zap.Error(err)) + return + } + serverInfo := version.ParseServerInfo(versionInfo) + version := semver.New(defaultSupportVectorVersion) + if m.cfg.IsTiDB && !serverInfo.ServerVersion.LessThan(*version) { + return + } + ddl.Query = formatQuery(ddl.Query) +} diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index 04c7da1b0a8..b0bf81938f0 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/cdc/sink/metrics" @@ -117,6 +118,15 @@ func TestPrepareDML(t *testing.T) { }, }, [][]int{{1, 2}}) + tableInfoVector := model.BuildTableInfo("common_1", "uk_without_pk", []*model.Column{ + nil, { + Name: "a1", + Type: mysql.TypeTiDBVectorFloat32, + }, { + Name: "a3", + Type: mysql.TypeTiDBVectorFloat32, + }}, [][]int{{}}) + testCases := []struct { input []*model.RowChangedEvent expected *preparedDMLs @@ -185,22 +195,25 @@ func TestPrepareDML(t *testing.T) { rowCount: 1, approximateSize: 63, }, - }, { + }, + // vector + { input: []*model.RowChangedEvent{ { - StartTs: 418658114257813518, - CommitTs: 418658114257813519, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, - Columns: []*model.Column{nil, { - Name: "a1", - Type: mysql.TypeTiDBVectorFloat32, - Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), - }, { - Name: "a3", - Type: mysql.TypeTiDBVectorFloat32, - Value: util.Must(types.ParseVectorFloat32("[1.1,-2.0,3.33,-4.12,-5]")), - }}, - IndexColumns: [][]int{{}}, + StartTs: 418658114257813518, + CommitTs: 418658114257813519, + TableInfo: tableInfoVector, + Columns: model.Columns2ColumnDatas( + []*model.Column{ + nil, { + Name: "a1", + Type: mysql.TypeTiDBVectorFloat32, + Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), + }, { + Name: "a3", + Type: mysql.TypeTiDBVectorFloat32, + Value: util.Must(types.ParseVectorFloat32("[1.1,-2.0,3.33,-4.12,-5]")), + }}, tableInfoVector), }, }, expected: &preparedDMLs{ diff --git a/cdc/sink/util/vector.go b/cdc/sink/util/vector.go index c5e5885f06e..80462fc011a 100644 --- a/cdc/sink/util/vector.go +++ b/cdc/sink/util/vector.go @@ -14,7 +14,7 @@ package util import ( - "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tiflow/pkg/errors" ) diff --git a/pkg/sink/codec/avro/avro_test.go b/pkg/sink/codec/avro/avro_test.go index fcf98e9cfc7..4be0ee03091 100644 --- a/pkg/sink/codec/avro/avro_test.go +++ b/pkg/sink/codec/avro/avro_test.go @@ -25,539 +25,36 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/sink/codec/utils" + "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/require" ) -func setupEncoderAndSchemaRegistry( - enableTiDBExtension bool, - decimalHandlingMode string, - bigintUnsignedHandlingMode string, -) (*BatchEncoder, error) { - startHTTPInterceptForTestingRegistry() - - keyManager, err := NewAvroSchemaManager( - context.Background(), - nil, - "http://127.0.0.1:8081", - "-key", - ) - if err != nil { - return nil, err - } - - valueManager, err := NewAvroSchemaManager( - context.Background(), - nil, - "http://127.0.0.1:8081", - "-value", - ) - if err != nil { - return nil, err - } +func TestDMLEventE2E(t *testing.T) { + codecConfig := common.NewConfig(config.ProtocolAvro) + codecConfig.EnableTiDBExtension = true + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() - return &BatchEncoder{ - namespace: model.DefaultNamespace, - valueSchemaManager: valueManager, - keySchemaManager: keyManager, - result: make([]*common.Message, 0, 1), - enableTiDBExtension: enableTiDBExtension, - decimalHandlingMode: decimalHandlingMode, - bigintUnsignedHandlingMode: bigintUnsignedHandlingMode, - }, nil -} + _, event, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) + colInfos := event.TableInfo.GetColInfosForRowChangedEvent() -func teardownEncoderAndSchemaRegistry() { - stopHTTPInterceptForTestingRegistry() -} + rand.New(rand.NewSource(time.Now().Unix())).Shuffle(len(event.Columns), func(i, j int) { + event.Columns[i], event.Columns[j] = event.Columns[j], event.Columns[i] + colInfos[i], colInfos[j] = colInfos[j], colInfos[i] + }) -type avroTestColumnTuple struct { - col model.Column - colInfo rowcodec.ColInfo - expectedSchema interface{} - expectedData interface{} - expectedType string -} + for _, decimalHandling := range []string{"precise", "string"} { + for _, unsignedBigintHandling := range []string{"long", "string"} { + codecConfig.AvroDecimalHandlingMode = decimalHandling + codecConfig.AvroBigintUnsignedHandlingMode = unsignedBigintHandling -var avroTestColumns = []*avroTestColumnTuple{ - { - model.Column{Name: "tiny", Value: int64(1), Type: mysql.TypeTiny}, - rowcodec.ColInfo{ - ID: 1, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeTiny), - }, - avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, - int32(1), "int", - }, - { - model.Column{Name: "short", Value: int64(1), Type: mysql.TypeShort}, - rowcodec.ColInfo{ - ID: 2, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeShort), - }, - avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, - int32(1), "int", - }, - { - model.Column{Name: "int24", Value: int64(1), Type: mysql.TypeInt24}, - rowcodec.ColInfo{ - ID: 3, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeInt24), - }, - avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, - int32(1), "int", - }, - { - model.Column{Name: "long", Value: int64(1), Type: mysql.TypeLong}, - rowcodec.ColInfo{ - ID: 4, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeLong), - }, - avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, - int32(1), "int", - }, - { - model.Column{Name: "longlong", Value: int64(1), Type: mysql.TypeLonglong}, - rowcodec.ColInfo{ - ID: 5, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeLonglong), - }, - avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "BIGINT"}}, - int64(1), "long", - }, - { - model.Column{ - Name: "tinyunsigned", - Value: uint64(1), - Type: mysql.TypeTiny, - Flag: model.UnsignedFlag, - }, - rowcodec.ColInfo{ - ID: 6, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeTiny)), - }, - avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, - int32(1), "int", - }, - { - model.Column{ - Name: "shortunsigned", - Value: uint64(1), - Type: mysql.TypeShort, - Flag: model.UnsignedFlag, - }, - rowcodec.ColInfo{ - ID: 7, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeShort)), - }, - avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, - int32(1), "int", - }, - { - model.Column{ - Name: "int24unsigned", - Value: uint64(1), - Type: mysql.TypeInt24, - Flag: model.UnsignedFlag, - }, - rowcodec.ColInfo{ - ID: 8, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeInt24)), - }, - avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, - int32(1), "int", - }, - { - model.Column{ - Name: "longunsigned", - Value: uint64(1), - Type: mysql.TypeLong, - Flag: model.UnsignedFlag, - }, - rowcodec.ColInfo{ - ID: 9, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeLong)), - }, - avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, - int64(1), "long", - }, - { - model.Column{ - Name: "longlongunsigned", - Value: uint64(1), - Type: mysql.TypeLonglong, - Flag: model.UnsignedFlag, - }, - rowcodec.ColInfo{ - ID: 10, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeLonglong)), - }, - avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "BIGINT UNSIGNED"}}, - int64(1), "long", - }, - { - model.Column{Name: "float", Value: float32(3.14), Type: mysql.TypeFloat}, - rowcodec.ColInfo{ - ID: 11, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeFloat), - }, - avroSchema{Type: "float", Parameters: map[string]string{"tidb_type": "FLOAT"}}, - float32(3.14), "float", - }, - { - model.Column{Name: "double", Value: float64(3.14), Type: mysql.TypeDouble}, - rowcodec.ColInfo{ - ID: 12, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeDouble), - }, - avroSchema{Type: "double", Parameters: map[string]string{"tidb_type": "DOUBLE"}}, - float64(3.14), "double", - }, - { - model.Column{Name: "bit", Value: uint64(683), Type: mysql.TypeBit}, - rowcodec.ColInfo{ - ID: 13, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeBit), - }, - avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BIT", "length": "1"}}, - []byte("\x02\xab"), "bytes", - }, - { - model.Column{Name: "decimal", Value: "129012.1230000", Type: mysql.TypeNewDecimal}, - rowcodec.ColInfo{ - ID: 14, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeNewDecimal), - }, - avroLogicalTypeSchema{ - avroSchema: avroSchema{ - Type: "bytes", - Parameters: map[string]string{"tidb_type": "DECIMAL"}, - }, - LogicalType: "decimal", - Precision: 10, - Scale: 0, - }, - big.NewRat(129012123, 1000), "bytes.decimal", - }, - { - model.Column{Name: "tinytext", Value: []byte("hello world"), Type: mysql.TypeTinyBlob}, - rowcodec.ColInfo{ - ID: 15, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeBlob), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, - "hello world", "string", - }, - { - model.Column{Name: "mediumtext", Value: []byte("hello world"), Type: mysql.TypeMediumBlob}, - rowcodec.ColInfo{ - ID: 16, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeMediumBlob), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, - "hello world", "string", - }, - { - model.Column{Name: "text", Value: []byte("hello world"), Type: mysql.TypeBlob}, - rowcodec.ColInfo{ - ID: 17, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeBlob), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, - "hello world", "string", - }, - { - model.Column{Name: "longtext", Value: []byte("hello world"), Type: mysql.TypeLongBlob}, - rowcodec.ColInfo{ - ID: 18, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeLongBlob), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, - "hello world", "string", - }, - { - model.Column{Name: "varchar", Value: []byte("hello world"), Type: mysql.TypeVarchar}, - rowcodec.ColInfo{ - ID: 19, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeVarchar), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, - "hello world", "string", - }, - { - model.Column{Name: "varstring", Value: []byte("hello world"), Type: mysql.TypeVarString}, - rowcodec.ColInfo{ - ID: 20, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeVarString), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, - "hello world", "string", - }, - { - model.Column{Name: "string", Value: []byte("hello world"), Type: mysql.TypeString}, - rowcodec.ColInfo{ - ID: 21, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeString), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, - "hello world", "string", - }, - { - model.Column{ - Name: "tinyblob", - Value: []byte("hello world"), - Type: mysql.TypeTinyBlob, - Flag: model.BinaryFlag, - }, - rowcodec.ColInfo{ - ID: 22, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), - }, - avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, - []byte("hello world"), "bytes", - }, - { - model.Column{ - Name: "mediumblob", - Value: []byte("hello world"), - Type: mysql.TypeMediumBlob, - Flag: model.BinaryFlag, - }, - rowcodec.ColInfo{ - ID: 23, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), - }, - avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, - []byte("hello world"), "bytes", - }, - { - model.Column{ - Name: "blob", - Value: []byte("hello world"), - Type: mysql.TypeBlob, - Flag: model.BinaryFlag, - }, - rowcodec.ColInfo{ - ID: 24, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), - }, - avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, - []byte("hello world"), "bytes", - }, - { - model.Column{ - Name: "longblob", - Value: []byte("hello world"), - Type: mysql.TypeLongBlob, - Flag: model.BinaryFlag, - }, - rowcodec.ColInfo{ - ID: 25, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), - }, - avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, - []byte("hello world"), "bytes", - }, - { - model.Column{ - Name: "varbinary", - Value: []byte("hello world"), - Type: mysql.TypeVarchar, - Flag: model.BinaryFlag, - }, - rowcodec.ColInfo{ - ID: 26, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), - }, - avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, - []byte("hello world"), "bytes", - }, - { - model.Column{ - Name: "varbinary1", - Value: []byte("hello world"), - Type: mysql.TypeVarString, - Flag: model.BinaryFlag, - }, - rowcodec.ColInfo{ - ID: 27, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), - }, - avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, - []byte("hello world"), "bytes", - }, - { - model.Column{ - Name: "binary", - Value: []byte("hello world"), - Type: mysql.TypeString, - Flag: model.BinaryFlag, - }, - rowcodec.ColInfo{ - ID: 28, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeString)), - }, - avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, - []byte("hello world"), "bytes", - }, - { - model.Column{Name: "enum", Value: uint64(1), Type: mysql.TypeEnum}, - rowcodec.ColInfo{ - ID: 29, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetElems(types.NewFieldType(mysql.TypeEnum), []string{"a,", "b"}), - }, - avroSchema{ - Type: "string", - Parameters: map[string]string{"tidb_type": "ENUM", "allowed": "a\\,,b"}, - }, - "a,", "string", - }, - { - model.Column{Name: "set", Value: uint64(1), Type: mysql.TypeSet}, - rowcodec.ColInfo{ - ID: 30, - IsPKHandle: false, - VirtualGenCol: false, - Ft: common.SetElems(types.NewFieldType(mysql.TypeSet), []string{"a,", "b"}), - }, - avroSchema{ - Type: "string", - Parameters: map[string]string{"tidb_type": "SET", "allowed": "a\\,,b"}, - }, - "a,", "string", - }, - { - model.Column{Name: "json", Value: `{"key": "value"}`, Type: mysql.TypeJSON}, - rowcodec.ColInfo{ - ID: 31, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeJSON), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "JSON"}}, - `{"key": "value"}`, "string", - }, - { - model.Column{Name: "date", Value: "2000-01-01", Type: mysql.TypeDate}, - rowcodec.ColInfo{ - ID: 32, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeDate), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "DATE"}}, - "2000-01-01", "string", - }, - { - model.Column{Name: "datetime", Value: "2015-12-20 23:58:58", Type: mysql.TypeDatetime}, - rowcodec.ColInfo{ - ID: 33, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeDatetime), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "DATETIME"}}, - "2015-12-20 23:58:58", "string", - }, - { - model.Column{Name: "timestamp", Value: "1973-12-30 15:30:00", Type: mysql.TypeTimestamp}, - rowcodec.ColInfo{ - ID: 34, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeTimestamp), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TIMESTAMP"}}, - "1973-12-30 15:30:00", "string", - }, - { - model.Column{Name: "time", Value: "23:59:59", Type: mysql.TypeDuration}, - rowcodec.ColInfo{ - ID: 35, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeDuration), - }, - avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TIME"}}, - "23:59:59", "string", - }, - { - model.Column{Name: "year", Value: int64(1970), Type: mysql.TypeYear}, - rowcodec.ColInfo{ - ID: 36, - IsPKHandle: false, - VirtualGenCol: false, - Ft: types.NewFieldType(mysql.TypeYear), - }, - avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "YEAR"}}, - int32(1970), "int", - }, -} + encoder, err := SetupEncoderAndSchemaRegistry4Testing(ctx, codecConfig) + require.NoError(t, err) + require.NotNil(t, encoder) -func TestColumnToAvroSchema(t *testing.T) { - for _, v := range avroTestColumns { - schema, err := columnToAvroSchema(&v.col, v.colInfo.Ft, "precise", "long") - require.NoError(t, err) - require.Equal(t, v.expectedSchema, schema) - if v.col.Name == "decimal" { - schema, err := columnToAvroSchema(&v.col, v.colInfo.Ft, "string", "long") + topic := "avro-test-topic" + err = encoder.AppendRowChangedEvent(ctx, topic, event, func() {}) require.NoError(t, err) messages := encoder.Build() diff --git a/pkg/sink/codec/canal/canal_entry_test.go b/pkg/sink/codec/canal/canal_entry_test.go index 4735fd0cfef..fa4179f1a42 100644 --- a/pkg/sink/codec/canal/canal_entry_test.go +++ b/pkg/sink/codec/canal/canal_entry_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/sink/codec/internal" - "github.com/pingcap/tiflow/pkg/util" canal "github.com/pingcap/tiflow/proto/canal" "github.com/stretchr/testify/require" "golang.org/x/text/encoding/charmap" @@ -36,10 +35,11 @@ func TestInsert(t *testing.T) { name varchar(32), tiny tinyint, comment text, - bb blob)` + bb blob + vec vector)` _ = helper.DDL2Event(sql) - event := helper.DML2Event(`insert into test.t values(1, "Bob", 127, "测试", "测试blob")`, "test", "t") + event := helper.DML2Event(`insert into test.t values(1, "Bob", 127, "测试", "测试blob", '[1,2,3,4,5]')`, "test", "t") codecConfig := common.NewConfig(config.ProtocolCanalJSON) builder := newCanalEntryBuilder(codecConfig) diff --git a/pkg/sink/codec/maxwell/maxwell_message.go b/pkg/sink/codec/maxwell/maxwell_message.go index c0bb8942457..a868d65f4fd 100644 --- a/pkg/sink/codec/maxwell/maxwell_message.go +++ b/pkg/sink/codec/maxwell/maxwell_message.go @@ -85,7 +85,7 @@ func rowChangeToMaxwellMsg(e *model.RowChangedEvent, onlyHandleKeyColumns bool) value.Old[colName] = string(v.Value.([]byte)) } case mysql.TypeTiDBVectorFloat32: - value.Old[v.Name] = v.Value.(types.VectorFloat32).String() + value.Old[colName] = v.Value.(types.VectorFloat32).String() default: value.Old[colName] = v.Value } @@ -135,8 +135,8 @@ func rowChangeToMaxwellMsg(e *model.RowChangedEvent, onlyHandleKeyColumns bool) } case mysql.TypeTiDBVectorFloat32: val := v.Value.(types.VectorFloat32).String() - if value.Old[v.Name] != val { - value.Old[v.Name] = val + if value.Old[colName] != val { + value.Old[colName] = val } default: if value.Data[colName] != v.Value { From 2283416e07adb998b3085de4d9603734530e10b5 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 15 Aug 2024 18:48:42 +0800 Subject: [PATCH 03/37] . --- cdc/sink/dmlsink/txn/mysql/mysql.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index 16a0088324e..0b6c43b7963 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/cdc/sink/metrics" From e3ec642039a388f12c52f666b2cc8d49851b3aa4 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 22 Aug 2024 16:50:20 +0800 Subject: [PATCH 04/37] add dep --- pkg/sink/codec/canal/canal_entry.go | 1 + pkg/sink/codec/craft/message_decoder.go | 1 + pkg/sink/codec/craft/message_encoder.go | 1 + pkg/sink/codec/internal/column.go | 1 + pkg/sink/codec/maxwell/maxwell_message.go | 3 ++- 5 files changed, 6 insertions(+), 1 deletion(-) diff --git a/pkg/sink/codec/canal/canal_entry.go b/pkg/sink/codec/canal/canal_entry.go index 9bf77f73a6b..a79ff0f5fd3 100644 --- a/pkg/sink/codec/canal/canal_entry.go +++ b/pkg/sink/codec/canal/canal_entry.go @@ -24,6 +24,7 @@ import ( mm "github.com/pingcap/tidb/pkg/parser/model" timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec/common" diff --git a/pkg/sink/codec/craft/message_decoder.go b/pkg/sink/codec/craft/message_decoder.go index 4a6fbec8f87..016f8fd74f2 100644 --- a/pkg/sink/codec/craft/message_decoder.go +++ b/pkg/sink/codec/craft/message_decoder.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/errors" pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" ) diff --git a/pkg/sink/codec/craft/message_encoder.go b/pkg/sink/codec/craft/message_encoder.go index 7f219092f19..8f887226bdc 100644 --- a/pkg/sink/codec/craft/message_encoder.go +++ b/pkg/sink/codec/craft/message_encoder.go @@ -19,6 +19,7 @@ import ( "unsafe" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" ) diff --git a/pkg/sink/codec/internal/column.go b/pkg/sink/codec/internal/column.go index 916d3dfa89d..e124e84bc9c 100644 --- a/pkg/sink/codec/internal/column.go +++ b/pkg/sink/codec/internal/column.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" "go.uber.org/zap" ) diff --git a/pkg/sink/codec/maxwell/maxwell_message.go b/pkg/sink/codec/maxwell/maxwell_message.go index a868d65f4fd..6b48f5dd0e7 100644 --- a/pkg/sink/codec/maxwell/maxwell_message.go +++ b/pkg/sink/codec/maxwell/maxwell_message.go @@ -18,6 +18,7 @@ import ( model2 "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec/internal" @@ -105,7 +106,7 @@ func rowChangeToMaxwellMsg(e *model.RowChangedEvent, onlyHandleKeyColumns bool) value.Data[colName] = string(v.Value.([]byte)) } case mysql.TypeTiDBVectorFloat32: - value.Data[v.Name] = v.Value.(types.VectorFloat32).String() + value.Data[colName] = v.Value.(types.VectorFloat32).String() default: value.Data[colName] = v.Value } From aca6b17210bcf484a3400bd3f22e8d35bfdbe75c Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 22 Aug 2024 20:07:18 +0800 Subject: [PATCH 05/37] update --- cdc/server/server.go | 2 +- cdc/sink/util/vector.go | 48 ++++++++++++++++++----------------------- 2 files changed, 22 insertions(+), 28 deletions(-) diff --git a/cdc/server/server.go b/cdc/server/server.go index 205848779c7..858b647ed55 100644 --- a/cdc/server/server.go +++ b/cdc/server/server.go @@ -215,7 +215,7 @@ func (s *server) setMemoryLimit() { } if conf.GcTunerMemoryThreshold > 0 { gctuner.EnableGOGCTuner.Store(true) - gctuner.Tuning(uint32(conf.GcTunerMemoryThreshold)) + gctuner.Tuning(conf.GcTunerMemoryThreshold) log.Info("enable gctuner, set memory limit", zap.Uint64("bytes", conf.GcTunerMemoryThreshold), zap.String("memory", humanize.IBytes(conf.GcTunerMemoryThreshold)), diff --git a/cdc/sink/util/vector.go b/cdc/sink/util/vector.go index 80462fc011a..8747c05cebf 100644 --- a/cdc/sink/util/vector.go +++ b/cdc/sink/util/vector.go @@ -13,31 +13,25 @@ package util -import ( - "github.com/pingcap/tidb/pkg/table" - - "github.com/pingcap/tiflow/pkg/errors" -) - // CheckVectorIndexForColumnModify checks Vector Index constraints for MODIFY COLUMN. -func CheckVectorIndexForColumnModify(oldCol *table.Column, newCol *table.Column) error { - if oldCol.VectorIndex == nil && newCol.VectorIndex == nil { - return nil - } - if oldCol.VectorIndex == nil && newCol.VectorIndex != nil { - return errors.Errorf("currently HNSW index can be only defined when creating the table") - } - if oldCol.VectorIndex != nil && newCol.VectorIndex == nil { - return errors.Errorf("currently HNSW index can not be removed") - } - if oldCol.FieldType.GetFlen() != newCol.FieldType.GetFlen() { - return errors.New("cannot modify vector column's dimention when HNSW index is defined") - } - if oldCol.FieldType.GetType() != newCol.FieldType.GetType() { - return errors.New("cannot modify column data type when HNSW index is defined") - } - if *(oldCol.VectorIndex) != *(newCol.VectorIndex) { - return errors.New("currently HNSW index cannot be modified") - } - return nil -} +// func CheckVectorIndexForColumnModify(oldCol *table.Column, newCol *table.Column) error { +// if oldCol.VectorIndex == nil && newCol.VectorIndex == nil { +// return nil +// } +// if oldCol.VectorIndex == nil && newCol.VectorIndex != nil { +// return errors.Errorf("currently HNSW index can be only defined when creating the table") +// } +// if oldCol.VectorIndex != nil && newCol.VectorIndex == nil { +// return errors.Errorf("currently HNSW index can not be removed") +// } +// if oldCol.FieldType.GetFlen() != newCol.FieldType.GetFlen() { +// return errors.New("cannot modify vector column's dimention when HNSW index is defined") +// } +// if oldCol.FieldType.GetType() != newCol.FieldType.GetType() { +// return errors.New("cannot modify column data type when HNSW index is defined") +// } +// if *(oldCol.VectorIndex) != *(newCol.VectorIndex) { +// return errors.New("currently HNSW index cannot be modified") +// } +// return nil +// } From 826a24d3ea19ba771a86cc186fd68e56ba3f5c55 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Mon, 26 Aug 2024 12:11:51 +0800 Subject: [PATCH 06/37] update --- cdc/sink/dmlsink/txn/mysql/mysql.go | 24 ++++++++++++++---------- cdc/sink/dmlsink/txn/mysql/mysql_test.go | 2 +- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index 58d8bc4bd79..95edffaddda 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -619,16 +619,8 @@ func (s *mysqlBackend) prepareDMLs() *preparedDMLs { callbacks = nil } - // decode vector - for i := 0; i < len(values); i++ { - value := values[i] - for j := 0; j < len(value); j++ { - switch value[j].(type) { - case types.VectorFloat32: - value[j] = value[j].(types.VectorFloat32).String() - } - } - } + // Convert vector to string + decodeVector(values) return &preparedDMLs{ startTs: startTs, sqls: sqls, @@ -877,6 +869,18 @@ func getSQLErrCode(err error) (errors.ErrCode, bool) { return errors.ErrCode(mysqlErr.Number), true } +func decodeVector(values [][]interface{}) { + for i := 0; i < len(values); i++ { + value := values[i] + for j := 0; j < len(value); j++ { + switch value[j].(type) { + case types.VectorFloat32: + value[j] = value[j].(types.VectorFloat32).String() + } + } + } +} + // Only for testing. func (s *mysqlBackend) setDMLMaxRetry(maxRetry uint64) { s.dmlMaxRetry = maxRetry diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index 5c940104ff9..6f5df6e1ee2 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -196,7 +196,7 @@ func TestPrepareDML(t *testing.T) { approximateSize: 63, }, }, - // vector + // vector type { input: []*model.RowChangedEvent{ { From d018cab5fb7e3d8ff61328109338370e45958d95 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Tue, 27 Aug 2024 14:49:54 +0800 Subject: [PATCH 07/37] update --- cdc/sink/ddlsink/mysql/format_ddl.go | 33 +++++--- cdc/sink/ddlsink/mysql/format_ddl_test.go | 43 ++++++++++ cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 42 ++++++---- cdc/sink/dmlsink/txn/mysql/dml.go | 5 ++ cdc/sink/dmlsink/txn/mysql/mysql.go | 15 ---- go.mod | 38 ++++----- go.sum | 82 +++++++++---------- pkg/sink/codec/avro/avro.go | 8 +- pkg/sink/codec/canal/canal_json_message.go | 1 + pkg/sink/codec/csv/csv_message.go | 10 +-- pkg/sink/codec/internal/column.go | 2 +- .../vector/conf/diff_config.toml | 30 +++++++ tests/integration_tests/vector/data/data.sql | 24 ++++++ tests/integration_tests/vector/run.sh | 45 ++++++++++ 14 files changed, 264 insertions(+), 114 deletions(-) create mode 100644 cdc/sink/ddlsink/mysql/format_ddl_test.go create mode 100644 tests/integration_tests/vector/conf/diff_config.toml create mode 100644 tests/integration_tests/vector/data/data.sql create mode 100755 tests/integration_tests/vector/run.sh diff --git a/cdc/sink/ddlsink/mysql/format_ddl.go b/cdc/sink/ddlsink/mysql/format_ddl.go index 79c1249c49b..9fe07c3c89f 100644 --- a/cdc/sink/ddlsink/mysql/format_ddl.go +++ b/cdc/sink/ddlsink/mysql/format_ddl.go @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Inc. +// Copyright 2024 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,25 +16,32 @@ package mysql import ( "bytes" + "github.com/pingcap/log" "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/pingcap/tidb/pkg/types" + "go.uber.org/zap" ) -type fingerprintVisitor struct{} - -func (f *fingerprintVisitor) Enter(n ast.Node) (node ast.Node, skipChildren bool) { - if v, ok := n.(*ast.ColumnNameExpr); ok { - if v.Name.OrigColName() == "VECTOR" { - v.SetType(types.NewFieldType(mysql.TypeVarString)) +type visiter struct{} + +func (f *visiter) Enter(n ast.Node) (node ast.Node, skipChildren bool) { + switch v := n.(type) { + case *ast.ColumnDef: + switch v.Tp.GetType() { + case mysql.TypeTiDBVectorFloat32: + v.Tp.SetType(mysql.TypeLongBlob) + v.Tp.SetCharset("") + v.Tp.SetCollate("") + v.Tp.SetFlen(-1) + v.Options = []*ast.ColumnOption{} // clear COMMENT } } return n, false } -func (f *fingerprintVisitor) Leave(n ast.Node) (node ast.Node, ok bool) { +func (f *visiter) Leave(n ast.Node) (node ast.Node, ok bool) { return n, true } @@ -42,14 +49,14 @@ func formatQuery(sql string) string { p := parser.New() stmt, err := p.ParseOneStmt(sql, "", "") if err != nil { - + log.Error("format query parse one stmt failed", zap.Error(err)) } - stmt.Accept(&fingerprintVisitor{}) + stmt.Accept(&visiter{}) buf := new(bytes.Buffer) restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, buf) - err = stmt.Restore(restoreCtx) - if nil != err { + if err = stmt.Restore(restoreCtx); err != nil { + log.Error("format query restore failed", zap.Error(err)) } return buf.String() } diff --git a/cdc/sink/ddlsink/mysql/format_ddl_test.go b/cdc/sink/ddlsink/mysql/format_ddl_test.go new file mode 100644 index 00000000000..beb5bc7e77b --- /dev/null +++ b/cdc/sink/ddlsink/mysql/format_ddl_test.go @@ -0,0 +1,43 @@ +// 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 mysql + +import ( + "bytes" + "testing" + + "github.com/pingcap/log" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/format" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +func TestFormatQuery(t *testing.T) { + sql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` VECTOR(5));" + expectSql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT)" + p := parser.New() + stmt, err := p.ParseOneStmt(sql, "", "") + if err != nil { + log.Error("format query parse one stmt failed", zap.Error(err)) + } + stmt.Accept(&visiter{}) + + buf := new(bytes.Buffer) + restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, buf) + if err = stmt.Restore(restoreCtx); err != nil { + log.Error("format query restore failed", zap.Error(err)) + } + require.Equal(t, buf.String(), expectSql) +} diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index e087528da25..3eaf38bc97f 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -46,7 +46,7 @@ const ( // networkDriftDuration is used to construct a context timeout for database operations. networkDriftDuration = 5 * time.Second - defaultSupportVectorVersion = "v8.3.0" + defaultSupportVectorVersion = "8.3.0" ) // GetDBConnImpl is the implementation of pmysql.IDBConnectionFactory. @@ -71,6 +71,8 @@ type DDLSink struct { // is running in downstream. // map: model.TableName -> timodel.ActionType lastExecutedNormalDDLCache *lru.Cache + + needFormat bool } // NewDDLSink creates a new DDLSink. @@ -110,12 +112,14 @@ func NewDDLSink( if err != nil { return nil, err } + m := &DDLSink{ id: changefeedID, db: db, cfg: cfg, statistics: metrics.NewStatistics(ctx, changefeedID, sink.TxnSink), lastExecutedNormalDDLCache: lruCache, + needFormat: needFormatDDL(db, cfg.IsTiDB), } log.Info("MySQL DDL sink is created", @@ -203,7 +207,11 @@ func (m *DDLSink) execDDL(pctx context.Context, ddl *model.DDLEvent) error { shouldSwitchDB := needSwitchDB(ddl) - m.formatQuery(ddl) + // Convert vector type to string type for unsupport database + if m.needFormat { + ddl.Query = formatQuery(ddl.Query) + log.Warn("format ddl query", zap.String("query", ddl.Query), zap.String("collate", ddl.Collate), zap.String("charset", ddl.Charset)) + } failpoint.Inject("MySQLSinkExecDDLDelay", func() { select { @@ -278,6 +286,22 @@ func needSwitchDB(ddl *model.DDLEvent) bool { return true } +// needFormatDDL checks vector type support +func needFormatDDL(db *sql.DB, isTiDB bool) bool { + versionInfo, err := export.SelectVersion(db) + if err != nil { + log.Warn("fail to get version", zap.Error(err), zap.Bool("isTiDB", isTiDB)) + return false + } + serverInfo := version.ParseServerInfo(versionInfo) + version := semver.New(defaultSupportVectorVersion) + if !isTiDB || serverInfo.ServerVersion.LessThan(*version) { + log.Error("downstream unsupport vector type. hack: we convert it to longtext", zap.String("support version", version.String()), zap.Bool("isTiDB", isTiDB)) + return true + } + return false +} + // WriteCheckpointTs does nothing. func (m *DDLSink) WriteCheckpointTs(_ context.Context, _ uint64, _ []*model.TableInfo) error { // Only for RowSink for now. @@ -298,17 +322,3 @@ func (m *DDLSink) Close() { } } } - -func (m *DDLSink) formatQuery(ddl *model.DDLEvent) { - versionInfo, err := export.SelectVersion(m.db) - if err != nil { - log.Error("fail to get version info", zap.Error(err)) - return - } - serverInfo := version.ParseServerInfo(versionInfo) - version := semver.New(defaultSupportVectorVersion) - if m.cfg.IsTiDB && !serverInfo.ServerVersion.LessThan(*version) { - return - } - ddl.Query = formatQuery(ddl.Query) -} diff --git a/cdc/sink/dmlsink/txn/mysql/dml.go b/cdc/sink/dmlsink/txn/mysql/dml.go index da29618908f..1bcbcc882ad 100644 --- a/cdc/sink/dmlsink/txn/mysql/dml.go +++ b/cdc/sink/dmlsink/txn/mysql/dml.go @@ -17,6 +17,7 @@ import ( "strings" "github.com/pingcap/tidb/pkg/parser/charset" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/quotes" ) @@ -117,6 +118,10 @@ func appendQueryArgs(args []interface{}, col *model.Column) []interface{} { args = append(args, col.Value) } } else { + // convert vector to string + if val, ok := col.Value.(types.VectorFloat32); ok { + col.Value = val.String() + } args = append(args, col.Value) } diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index 95edffaddda..ac11a198014 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" - "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/cdc/sink/metrics" @@ -619,8 +618,6 @@ func (s *mysqlBackend) prepareDMLs() *preparedDMLs { callbacks = nil } - // Convert vector to string - decodeVector(values) return &preparedDMLs{ startTs: startTs, sqls: sqls, @@ -869,18 +866,6 @@ func getSQLErrCode(err error) (errors.ErrCode, bool) { return errors.ErrCode(mysqlErr.Number), true } -func decodeVector(values [][]interface{}) { - for i := 0; i < len(values); i++ { - value := values[i] - for j := 0; j < len(value); j++ { - switch value[j].(type) { - case types.VectorFloat32: - value[j] = value[j].(types.VectorFloat32).String() - } - } - } -} - // Only for testing. func (s *mysqlBackend) setDMLMaxRetry(maxRetry uint64) { s.dmlMaxRetry = maxRetry diff --git a/go.mod b/go.mod index 3757a1ef1dd..109638e1a3b 100644 --- a/go.mod +++ b/go.mod @@ -67,11 +67,11 @@ require ( github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 - github.com/pingcap/kvproto v0.0.0-20240620063548-118a4cab53e4 + github.com/pingcap/kvproto v0.0.0-20240716095229-5f7ffec83ea7 github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d - github.com/pingcap/tidb v1.1.0-beta.0.20240722024203-504960d51b2a + github.com/pingcap/tidb v1.1.0-beta.0.20240822082843-d2b20bd898d1 github.com/pingcap/tidb-dashboard v0.0.0-20240326110213-9768844ff5d7 - github.com/pingcap/tidb/pkg/parser v0.0.0-20240722015532-8edd4ed54376 + github.com/pingcap/tidb/pkg/parser v0.0.0-20240822082843-d2b20bd898d1 github.com/prometheus/client_golang v1.19.1 github.com/prometheus/client_model v0.6.1 github.com/r3labs/diff v1.1.0 @@ -89,9 +89,9 @@ require ( github.com/swaggo/swag v1.16.3 github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 github.com/thanhpk/randstr v1.0.6 - github.com/tikv/client-go/v2 v2.0.8-0.20240703095801-d73cc1ed6503 + github.com/tikv/client-go/v2 v2.0.8-0.20240815020919-c810ed88fb02 github.com/tikv/pd v1.1.0-beta.0.20240407022249-7179657d129b - github.com/tikv/pd/client v0.0.0-20240717053728-5ec6af403019 + github.com/tikv/pd/client v0.0.0-20240805092608-838ee7983b78 github.com/tinylib/msgp v1.1.6 github.com/uber-go/atomic v1.4.0 github.com/vmihailenco/msgpack/v5 v5.3.5 @@ -111,19 +111,19 @@ require ( go.uber.org/ratelimit v0.2.0 go.uber.org/zap v1.27.0 golang.org/x/exp v0.0.0-20240604190554-fc45aab8b7f8 - golang.org/x/net v0.27.0 + golang.org/x/net v0.28.0 golang.org/x/oauth2 v0.21.0 - golang.org/x/sync v0.7.0 - golang.org/x/sys v0.22.0 - golang.org/x/text v0.16.0 + golang.org/x/sync v0.8.0 + golang.org/x/sys v0.24.0 + golang.org/x/text v0.17.0 golang.org/x/time v0.5.0 google.golang.org/genproto/googleapis/api v0.0.0-20240401170217-c3f982113cda google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 google.golang.org/grpc v1.64.0 google.golang.org/protobuf v1.34.2 gopkg.in/yaml.v2 v2.4.0 - gorm.io/driver/mysql v1.4.5 - gorm.io/gorm v1.24.5 + gorm.io/driver/mysql v1.5.7 + gorm.io/gorm v1.25.11 upper.io/db.v3 v3.7.1+incompatible ) @@ -309,7 +309,7 @@ require ( 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/mattn/go-runewidth v0.0.15 // indirect + github.com/mattn/go-runewidth v0.0.16 // indirect github.com/mattn/go-sqlite3 v2.0.1+incompatible // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/mtibben/percent v0.2.1 // indirect @@ -321,14 +321,14 @@ require ( github.com/opentracing/basictracer-go v1.1.0 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pelletier/go-toml/v2 v2.0.8 // indirect - github.com/petermattis/goid v0.0.0-20231207134359-e60b3f734c67 // indirect + github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7 // indirect github.com/philhofer/fwd v1.1.1 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d // indirect github.com/pingcap/fn v1.0.0 // indirect github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 // indirect github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 - github.com/pingcap/tipb v0.0.0-20240318032315-55a7867ddd50 // indirect + github.com/pingcap/tipb v0.0.0-20240703084358-e46e4632bd2b // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect @@ -339,7 +339,7 @@ require ( github.com/rivo/uniseg v0.4.7 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/rs/cors v1.7.0 // indirect - github.com/sasha-s/go-deadlock v0.3.1 // indirect + github.com/sasha-s/go-deadlock v0.3.5 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/shurcooL/httpfs v0.0.0-20230704072500-f1e31cf0ba5c // indirect github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 // indirect @@ -380,10 +380,10 @@ require ( go.opentelemetry.io/otel/sdk v1.24.0 // indirect go.opentelemetry.io/otel/trace v1.24.0 // indirect go.opentelemetry.io/proto/otlp v1.1.0 // indirect - golang.org/x/crypto v0.25.0 // indirect - golang.org/x/mod v0.18.0 // indirect - golang.org/x/term v0.22.0 - golang.org/x/tools v0.22.0 // indirect + golang.org/x/crypto v0.26.0 // indirect + golang.org/x/mod v0.20.0 // indirect + golang.org/x/term v0.23.0 + golang.org/x/tools v0.24.0 // indirect google.golang.org/api v0.170.0 // indirect gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect diff --git a/go.sum b/go.sum index dd74ef9a83f..bd0d0709037 100644 --- a/go.sum +++ b/go.sum @@ -425,7 +425,7 @@ github.com/go-resty/resty/v2 v2.11.0 h1:i7jMfNOJYMp69lq7qozJP+bjgzfAzeOhuGlyDrqx github.com/go-resty/resty/v2 v2.11.0/go.mod h1:iiP/OpA0CkcL3IGt1O0+/SIItFUbkkyw5BGXiVdTu+A= github.com/go-session/session v3.1.2+incompatible/go.mod h1:8B3iivBQjrz/JtC68Np2T1yBBLxTan3mn/3OM0CyRt0= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= -github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= +github.com/go-sql-driver/mysql v1.7.0/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= 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= @@ -637,7 +637,6 @@ github.com/jfcg/sorty/v2 v2.1.0/go.mod h1:JpcSKlmtGOOAGyTdWN2ErjvxeMSJVYBsylAKep github.com/jhump/protoreflect v1.6.0/go.mod h1:eaTn3RZAmMBcV0fifFvlm6VHNz3wSkYyXYWUh7ymB74= github.com/jinzhu/inflection v1.0.0 h1:K317FqzuhWc8YvSVlFMCCUb36O/S9MCKRDI7QkRKD/E= github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= -github.com/jinzhu/now v1.1.4/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= @@ -773,8 +772,8 @@ github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= -github.com/mattn/go-runewidth v0.0.15 h1:UNAjwbU9l54TA3KzvqLGxwWjHmMgBUVhBiTjelZgg3U= -github.com/mattn/go-runewidth v0.0.15/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= +github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= +github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-shellwords v1.0.12 h1:M2zGm7EW6UQJvDeQxo4T51eKPurbeFbe8WtebGE2xrk= github.com/mattn/go-shellwords v1.0.12/go.mod h1:EZzvwXDESEeg03EKmM+RmDnNOPKG4lLtQsUlTZDWQ8Y= github.com/mattn/go-sqlite3 v1.14.6/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= @@ -860,9 +859,8 @@ github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwp github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml/v2 v2.0.8 h1:0ctb6s9mE31h0/lhu+J6OPmVeDxJn+kYnJc2jZR9tGQ= github.com/pelletier/go-toml/v2 v2.0.8/go.mod h1:vuYfssBdrU2XDZ9bYydBu6t+6a6PYNcZljzZR9VXg+4= -github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= -github.com/petermattis/goid v0.0.0-20231207134359-e60b3f734c67 h1:jik8PHtAIsPlCRJjJzl4udgEf7hawInF9texMeO2jrU= -github.com/petermattis/goid v0.0.0-20231207134359-e60b3f734c67/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= +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/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= github.com/philhofer/fwd v1.1.1 h1:GdGcTjf5RNAxwS4QLsiMzJYj5KEvPJD3Abr261yRQXQ= @@ -892,8 +890,8 @@ github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z 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-20240620063548-118a4cab53e4 h1:6aIKNB2YGAec4IUDLw6G2eDECiGiufZcgEbZSCELBx0= -github.com/pingcap/kvproto v0.0.0-20240620063548-118a4cab53e4/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= +github.com/pingcap/kvproto v0.0.0-20240716095229-5f7ffec83ea7 h1:V9XS3FQ/P6u+kFaoSyY5DBswIA774BMpIOLDBMrpxKc= +github.com/pingcap/kvproto v0.0.0-20240716095229-5f7ffec83ea7/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= @@ -901,14 +899,14 @@ github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d h1:y3EueKVfVykdpTyfU github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d/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.20240722024203-504960d51b2a h1:lIj1fFRvB1EwU5005B7vlAl5eu1LUg18FTw83X5Ed1A= -github.com/pingcap/tidb v1.1.0-beta.0.20240722024203-504960d51b2a/go.mod h1:2FO9Yzc8HOYk+aZRYrguuwN6OS12TE+OA2RtukAS+Z8= +github.com/pingcap/tidb v1.1.0-beta.0.20240822082843-d2b20bd898d1 h1:xsXvpphP/G3VVzicD5sU5iA5OBkLYxcGAjR9KmhIziE= +github.com/pingcap/tidb v1.1.0-beta.0.20240822082843-d2b20bd898d1/go.mod h1:fIHzhGkdtFsq29vDWpVLQZAUDCQYc5Khd3UF8J+vxd0= github.com/pingcap/tidb-dashboard v0.0.0-20240326110213-9768844ff5d7 h1:eFu98FbfJB7PKWOtkaV6YNXXJWqDhczQX56j/iucgU4= github.com/pingcap/tidb-dashboard v0.0.0-20240326110213-9768844ff5d7/go.mod h1:ucZBRz52icb23T/5Z4CsuUHmarYiin7p2MeiVBe+o8c= -github.com/pingcap/tidb/pkg/parser v0.0.0-20240722015532-8edd4ed54376 h1:tz757zpST60fweJB5ZjVl74kQGbRrpmIiTtF6BdnMow= -github.com/pingcap/tidb/pkg/parser v0.0.0-20240722015532-8edd4ed54376/go.mod h1:c/4la2yfv1vBYvtIG8WCDyDinLMDIUC5+zLRHiafY+Y= -github.com/pingcap/tipb v0.0.0-20240318032315-55a7867ddd50 h1:fVNBE06Rjec+EIHaYAKAHa/bIt5lnu3Zh9O6kV7ZAdg= -github.com/pingcap/tipb v0.0.0-20240318032315-55a7867ddd50/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tidb/pkg/parser v0.0.0-20240822082843-d2b20bd898d1 h1:AdP6/kzxPTEXFjWCDl4beuRKo7lrOLW2kjWVccAOx74= +github.com/pingcap/tidb/pkg/parser v0.0.0-20240822082843-d2b20bd898d1/go.mod h1:c/4la2yfv1vBYvtIG8WCDyDinLMDIUC5+zLRHiafY+Y= +github.com/pingcap/tipb v0.0.0-20240703084358-e46e4632bd2b h1:tySAGYw21A3Xa8CcA9jBTfrgAB3+KQWyqyW7fUyokzk= +github.com/pingcap/tipb v0.0.0-20240703084358-e46e4632bd2b/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -974,8 +972,8 @@ github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfF github.com/ruudk/golang-pdf417 v0.0.0-20201230142125-a7e3863a1245/go.mod h1:pQAZKsJ8yyVxGRWYNEm9oFB8ieLgKFnamEyDmSA0BRk= github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46 h1:GHRpF1pTW19a8tTFrMLUcfWwyC0pnifVo2ClaLq+hP8= github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5PCi+MFsC7HjREoAz1BU+Mq60+05gifQSsHSDG/8= -github.com/sasha-s/go-deadlock v0.3.1 h1:sqv7fDNShgjcaxkO0JNcOAlr8B9+cV5Ey/OB71efZx0= -github.com/sasha-s/go-deadlock v0.3.1/go.mod h1:F73l+cr82YSh10GxyRI6qZiCgK64VaZjwesgfQ1/iLM= +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/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= @@ -1104,12 +1102,12 @@ github.com/tidwall/rtree v0.0.0-20180113144539-6cd427091e0e/go.mod h1:/h+UnNGt0I github.com/tidwall/tinyqueue v0.0.0-20180302190814-1e39f5511563/go.mod h1:mLqSmt7Dv/CNneF2wfcChfN1rvapyQr01LGKnKex0DQ= github.com/tidwall/tinyqueue v0.1.1 h1:SpNEvEggbpyN5DIReaJ2/1ndroY8iyEGxPYxoSaymYE= github.com/tidwall/tinyqueue v0.1.1/go.mod h1:O/QNHwrnjqr6IHItYrzoHAKYhBkLI67Q096fQP5zMYw= -github.com/tikv/client-go/v2 v2.0.8-0.20240703095801-d73cc1ed6503 h1:0mUlg3+dA5LvwKs1U6i/ID/8RsYgLVLGyM8fSBMb630= -github.com/tikv/client-go/v2 v2.0.8-0.20240703095801-d73cc1ed6503/go.mod h1:4HDOAx8OXAJPtqhCZ03IhChXgaFs4B3+vSrPWmiPxjg= +github.com/tikv/client-go/v2 v2.0.8-0.20240815020919-c810ed88fb02 h1:XKZTb6ZyosZSkvOlmROlhGVHlGHEa3FmIip86cRI1TY= +github.com/tikv/client-go/v2 v2.0.8-0.20240815020919-c810ed88fb02/go.mod h1:4HDOAx8OXAJPtqhCZ03IhChXgaFs4B3+vSrPWmiPxjg= github.com/tikv/pd v1.1.0-beta.0.20240407022249-7179657d129b h1:t2XoZp4UHrkPpYPsxbRTRVExJnriWlh+ZsDIfpYyd98= github.com/tikv/pd v1.1.0-beta.0.20240407022249-7179657d129b/go.mod h1:7HJMdb0O5umNpZIFt8e/wKAcEmH99n2HsYgXX+vZj3k= -github.com/tikv/pd/client v0.0.0-20240717053728-5ec6af403019 h1:7VoatJKzIrsjepOaXQjpAcgxQrx2QBAI4HuZ0wFdinA= -github.com/tikv/pd/client v0.0.0-20240717053728-5ec6af403019/go.mod h1:QeMzXKDOW+GbbE+ckcVPBVS6vX3//QB99dXU+niYRq0= +github.com/tikv/pd/client v0.0.0-20240805092608-838ee7983b78 h1:PtW+yTvs9eGTMblulaCHmJ5OtifuE4SJXCACCtkd6ko= +github.com/tikv/pd/client v0.0.0-20240805092608-838ee7983b78/go.mod h1:TxrJRY949Vl14Lmarx6hTNP/HEDYzn4dP0KmjdzQ59w= github.com/tinylib/msgp v1.1.6 h1:i+SbKraHhnrf9M5MYmvQhFnbLhAXSDWF8WWsuyRdocw= github.com/tinylib/msgp v1.1.6/go.mod h1:75BAfg2hauQhs3qedfdDZmWAPcFMAvJE5b9rGOMufyw= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= @@ -1297,8 +1295,8 @@ golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= -golang.org/x/crypto v0.25.0 h1:ypSNr+bnYL2YhwoMt2zPxHFmbAN1KZs/njMG3hxUp30= -golang.org/x/crypto v0.25.0/go.mod h1:T+wALwcMOSE0kXgUAnPAHqTLW+XHgcELELW8VaDgm/M= +golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= +golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1354,8 +1352,8 @@ golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.18.0 h1:5+9lSbEzPSdWkH32vYPBwEpX8KwDbM52Ud9xBUvNlb0= -golang.org/x/mod v0.18.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.20.0 h1:utOm6MM3R3dnawAiJgn0y+xvuYRsm1RKM/4giyfDgV0= +golang.org/x/mod v0.20.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180530234432-1e491301e022/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= 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= @@ -1399,8 +1397,8 @@ golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= 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.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1419,8 +1417,8 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/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.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180816055513-1c9583448a9c/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= 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= @@ -1495,8 +1493,8 @@ 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.19.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= 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= @@ -1505,8 +1503,8 @@ golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= 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/term v0.22.0 h1:BbsgPEJULsl2fV/AT3v15Mjva5yXKQDyKf+TbDz7QJk= -golang.org/x/term v0.22.0/go.mod h1:F3qCibpT5AMpCRfhfT53vVJwhLtIVHhB9XDjfFvnMI4= +golang.org/x/term v0.23.0 h1:F6D4vR+EHoL9/sWAWgAR1H2DcHr4PareCbAaCo1RpuU= +golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1521,8 +1519,8 @@ golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= 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.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1586,8 +1584,8 @@ golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= 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.22.0 h1:gqSGLZqv+AI9lIQzniJ0nZDRG5GBPsSi+DRNHWNz6yA= -golang.org/x/tools v0.22.0/go.mod h1:aCwcsjqvq7Yqt6TNyX7QMU2enbQ/Gt0bo6krSeEri+c= +golang.org/x/tools v0.24.0 h1:J1shsA93PJUEVaUSaay7UXAyE8aimq3GW0pjlolpa24= +golang.org/x/tools v0.24.0/go.mod h1:YhNqVBIfWHdzvTLs0d8LCuMhkKUgSUKldakyV7W/WDQ= 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= @@ -1730,11 +1728,11 @@ gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C gopkg.in/yaml.v3 v3.0.0-20220512140231-539c8e751b99/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= -gorm.io/driver/mysql v1.3.3 h1:jXG9ANrwBc4+bMvBcSl8zCfPBaVoPyBEBshA8dA93X8= -gorm.io/driver/mysql v1.3.3/go.mod h1:ChK6AHbHgDCFZyJp0F+BmVGb06PSIoh9uVYKAlRbb2U= -gorm.io/gorm v1.23.1/go.mod h1:l2lP/RyAtc1ynaTjFksBde/O8v9oOGIApu2/xRitmZk= -gorm.io/gorm v1.24.5 h1:g6OPREKqqlWq4kh/3MCQbZKImeB9e6Xgc4zD+JgNZGE= -gorm.io/gorm v1.24.5/go.mod h1:DVrVomtaYTbqs7gB/x2uVvqnXzv0nqjB396B8cG4dBA= +gorm.io/driver/mysql v1.5.7 h1:MndhOPYOfEp2rHKgkZIhJ16eVUIRf2HmzgoPmh7FCWo= +gorm.io/driver/mysql v1.5.7/go.mod h1:sEtPWMiqiN1N1cMXoXmBbd8C6/l+TESwriotuRRpkDM= +gorm.io/gorm v1.25.7/go.mod h1:hbnx/Oo0ChWMn1BIhpy1oYozzpM15i4YPuHDmfYtwg8= +gorm.io/gorm v1.25.11 h1:/Wfyg1B/je1hnDx3sMkX+gAlxrlZpn6X0BXRlwXlvHg= +gorm.io/gorm v1.25.11/go.mod h1:xh7N7RHfYlNc5EmcI/El95gXusucDrQnHXe0+CgWcLQ= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/pkg/sink/codec/avro/avro.go b/pkg/sink/codec/avro/avro.go index 9847464bcb4..2b22d68689b 100644 --- a/pkg/sink/codec/avro/avro.go +++ b/pkg/sink/codec/avro/avro.go @@ -440,6 +440,8 @@ func mysqlTypeFromTiDBType(tidbType string) byte { result = mysql.TypeDuration case "YEAR": result = mysql.TypeYear + case "TiDBVECTORFloat32": + result = mysql.TypeTiDBVectorFloat32 default: log.Panic("this should not happen, unknown TiDB type", zap.String("type", tidbType)) } @@ -979,8 +981,10 @@ func (a *BatchEncoder) columnToAvroData( } return int32(col.Value.(int64)), "int", nil case mysql.TypeTiDBVectorFloat32: - vec := col.Value.(types.VectorFloat32) - return vec.String(), "string", nil + if vec, ok := col.Value.(types.VectorFloat32); ok { + return vec.String(), "string", nil + } + return nil, "", cerror.ErrAvroEncodeFailed default: log.Error("unknown mysql type", zap.Any("value", col.Value), zap.Any("mysqlType", col.Type)) return nil, "", cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") diff --git a/pkg/sink/codec/canal/canal_json_message.go b/pkg/sink/codec/canal/canal_json_message.go index 3072af553dd..64e7b438188 100644 --- a/pkg/sink/codec/canal/canal_json_message.go +++ b/pkg/sink/codec/canal/canal_json_message.go @@ -287,6 +287,7 @@ func canalJSONFormatColumn(value interface{}, name string, mysqlTypeStr string) if err != nil { log.Panic("invalid column value for double", zap.Any("col", result), zap.Error(err)) } + case mysql.TypeTiDBVectorFloat32: } result.Value = value diff --git a/pkg/sink/codec/csv/csv_message.go b/pkg/sink/codec/csv/csv_message.go index 36bfae82219..087daa9ce8a 100644 --- a/pkg/sink/codec/csv/csv_message.go +++ b/pkg/sink/codec/csv/csv_message.go @@ -311,9 +311,6 @@ func fromCsvValToColValue(csvConfig *common.Config, csvVal any, ft types.FieldTy case mysql.TypeBit: val, err := strconv.ParseUint(str, 10, 64) return val, err - case mysql.TypeTiDBVectorFloat32: - vec, err := types.ParseVectorFloat32(str) - return vec.String(), err default: return str, nil } @@ -366,8 +363,10 @@ func fromColValToCsvVal(csvConfig *common.Config, col *model.Column, ft *types.F } return setVar.Name, nil case mysql.TypeTiDBVectorFloat32: - vec := col.Value.(types.VectorFloat32) - return vec.String(), nil + if vec, ok := col.Value.(types.VectorFloat32); ok { + return vec.String(), nil + } + return nil, cerror.ErrCSVEncodeFailed default: return col.Value, nil } @@ -446,7 +445,6 @@ func csvMsg2RowChangedEvent(csvConfig *common.Config, csvMsg *csvMessage, tableI if err != nil { return nil, err } - return e, nil } diff --git a/pkg/sink/codec/internal/column.go b/pkg/sink/codec/internal/column.go index e124e84bc9c..0070f3f138c 100644 --- a/pkg/sink/codec/internal/column.go +++ b/pkg/sink/codec/internal/column.go @@ -101,8 +101,8 @@ func (c *Column) ToRowChangeColumn(name string) *model.Column { zap.Any("col", c), zap.Error(err)) } col.Value = uint64(val) + case mysql.TypeTiDBVectorFloat32: default: - col.Value = c.Value } return col } diff --git a/tests/integration_tests/vector/conf/diff_config.toml b/tests/integration_tests/vector/conf/diff_config.toml new file mode 100644 index 00000000000..d37f42f32c8 --- /dev/null +++ b/tests/integration_tests/vector/conf/diff_config.toml @@ -0,0 +1,30 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/vector/sync_diff/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/vector/data/data.sql b/tests/integration_tests/vector/data/data.sql new file mode 100644 index 00000000000..86371f2d7c5 --- /dev/null +++ b/tests/integration_tests/vector/data/data.sql @@ -0,0 +1,24 @@ +DROP DATABASE IF EXISTS test; +CREATE DATABASE test; +use test; +DROP table IF EXISTS test.simple1; +DROP table IF EXISTS test.simple2; + +CREATE table test.simple1(id int primary key, data VECTOR(5)); +-- CREATE VECTOR INDEX idx_name1 USING HNSW ON test.simple1(VEC_COSINE_DISTANCE(data)) ; +INSERT INTO test.simple1(id, data) VALUES (1, "[1,2,3,4,5]"); +INSERT INTO test.simple1(id, data) VALUES (2, '[2,3,4,5,6]'); +INSERT INTO test.simple1(id, data) VALUES (3, '[0.1,0.2,0.3,0.4,0.5]'); +INSERT INTO test.simple1(id, data) VALUES (4, '[0,-0.1,-2,2,0.1]'); + + +CREATE table test.simple2(id int primary key, data VECTOR(5), embedding VECTOR(5) COMMENT "hnsw(distance=cosine)"); +INSERT INTO test.simple2(id, data, embedding) VALUES (1, '[1,2,3,4,5]','[1,2,3,4,5]'); +INSERT INTO test.simple2(id, data, embedding) VALUES (2, '[2,3,4,5,6]','[1,2,3,4,5]'); +INSERT INTO test.simple2(id, data, embedding) VALUES (3, '[0.1,0.2,0.3,0.4,0.5]','[1,2,3,4,5]'); +INSERT INTO test.simple2(id, data, embedding) VALUES (4, '[0,-0.1,-2,2,0.1]','[1,2,3,4,5]'); + +DELETE FROM test.simple1 where id=1; +DELETE FROM test.simple2 where id=1; +DELETE FROM test.simple1 where id=2; +DELETE FROM test.simple2 where id=2; diff --git a/tests/integration_tests/vector/run.sh b/tests/integration_tests/vector/run.sh new file mode 100755 index 00000000000..87a94685fb3 --- /dev/null +++ b/tests/integration_tests/vector/run.sh @@ -0,0 +1,45 @@ +#!/bin/bash + +set -eu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +function run() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + TOPIC_NAME="vector-$RANDOM" + case $SINK_TYPE in + kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; + storage) SINK_URI="file://$WORK_DIR/storage_test/$TOPIC_NAME?protocol=canal-json&enable-tidb-extension=true" ;; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/" ;; + esac + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" + case $SINK_TYPE in + kafka) run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; + storage) run_storage_consumer $WORK_DIR $SINK_URI "" "" ;; + esac + run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + sleep 3 + # sync_diff can't check non-exist table, so we check expected tables are created in downstream first + check_table_exists test.simple1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.simple2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + 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! >>>>>>" From 352c435b69157870043caeda60bfc9694f822d82 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 29 Aug 2024 13:28:11 +0800 Subject: [PATCH 08/37] fmt --- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 2 +- cdc/sink/dmlsink/txn/mysql/mysql_test.go | 6 ++++-- pkg/sink/codec/internal/java.go | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 3eaf38bc97f..377739a7421 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -296,7 +296,7 @@ func needFormatDDL(db *sql.DB, isTiDB bool) bool { serverInfo := version.ParseServerInfo(versionInfo) version := semver.New(defaultSupportVectorVersion) if !isTiDB || serverInfo.ServerVersion.LessThan(*version) { - log.Error("downstream unsupport vector type. hack: we convert it to longtext", zap.String("support version", version.String()), zap.Bool("isTiDB", isTiDB)) + log.Error("downstream unsupport vector type. we convert it to longtext", zap.String("supportVersion", version.String()), zap.Bool("isTiDB", isTiDB)) return true } return false diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index 6f5df6e1ee2..f3db2e18373 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -125,7 +125,8 @@ func TestPrepareDML(t *testing.T) { }, { Name: "a3", Type: mysql.TypeTiDBVectorFloat32, - }}, [][]int{{}}) + }, + }, [][]int{{}}) testCases := []struct { input []*model.RowChangedEvent @@ -213,7 +214,8 @@ func TestPrepareDML(t *testing.T) { Name: "a3", Type: mysql.TypeTiDBVectorFloat32, Value: util.Must(types.ParseVectorFloat32("[1.1,-2.0,3.33,-4.12,-5]")), - }}, tableInfoVector), + }, + }, tableInfoVector), }, }, expected: &preparedDMLs{ diff --git a/pkg/sink/codec/internal/java.go b/pkg/sink/codec/internal/java.go index 426f537c04a..b25421fb6ba 100644 --- a/pkg/sink/codec/internal/java.go +++ b/pkg/sink/codec/internal/java.go @@ -48,7 +48,7 @@ const ( // JavaSQLTypeJAVA_OBJECT JavaSQLType = 2000 // JavaSQLTypeDISTINCT JavaSQLType = 2001 // JavaSQLTypeSTRUCT JavaSQLType = 2002 - // JavaSQLTypeARRAY JavaSQLType = 2003 + // JavaSQLTypeARRAY JavaSQLType = 2003 // JavaSQLTypeREF JavaSQLType = 2006 // JavaSQLTypeDATALINK JavaSQLType = 70 // JavaSQLTypeBOOLEAN JavaSQLType = 16 From 7686261a5e4ba030a3e9040dcd0799a9527deea0 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 30 Aug 2024 15:41:53 +0800 Subject: [PATCH 09/37] fix --- cdc/sink/dmlsink/txn/mysql/mysql.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index ac11a198014..8cff20985a4 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/cdc/sink/metrics" @@ -467,6 +468,15 @@ func (s *mysqlBackend) batchSingleTxnDmls( } } + // convert vector to string + for _, value := range values { + for i, val := range value { + switch v := val.(type) { + case types.VectorFloat32: + value[i] = v.String() + } + } + } return } From d6d5df60000472cd9e9491c80a779c15f3c3f962 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 30 Aug 2024 18:27:53 +0800 Subject: [PATCH 10/37] add test --- cdc/sink/dmlsink/txn/mysql/mysql_test.go | 82 +++++++++++++++++++++++- 1 file changed, 80 insertions(+), 2 deletions(-) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index f3db2e18373..fe853e408ce 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -213,7 +213,7 @@ func TestPrepareDML(t *testing.T) { }, { Name: "a3", Type: mysql.TypeTiDBVectorFloat32, - Value: util.Must(types.ParseVectorFloat32("[1.1,-2.0,3.33,-4.12,-5]")), + Value: util.Must(types.ParseVectorFloat32("[1.1,-2,3.33,-4.12,-5]")), }, }, tableInfoVector), }, @@ -221,7 +221,7 @@ func TestPrepareDML(t *testing.T) { expected: &preparedDMLs{ startTs: []model.Ts{418658114257813518}, sqls: []string{"REPLACE INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)"}, - values: [][]interface{}{{"[1,2,3,4,5]", "[1.1,-2.0,3.33,-4.12,-5]"}}, + values: [][]interface{}{{"[1,2,3,4,5]", "[1.1,-2,3.33,-4.12,-5]"}}, rowCount: 1, approximateSize: 64, }, @@ -1152,6 +1152,14 @@ func TestPrepareBatchDMLs(t *testing.T) { Charset: charset.CharsetGBK, Flag: model.MultipleKeyFlag | model.HandleKeyFlag | model.UniqueKeyFlag, }}, [][]int{{0, 1}}) + tableInfoWithVector := model.BuildTableInfo("common_1", "uk_without_pk", []*model.Column{{ + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag | model.UniqueKeyFlag, + }, { + Name: "a3", + Type: mysql.TypeTiDBVectorFloat32, + }}, [][]int{{0, 1}}) testCases := []struct { isTiDB bool input []*model.RowChangedEvent @@ -1466,6 +1474,76 @@ func TestPrepareBatchDMLs(t *testing.T) { approximateSize: 204, }, }, + + // inser vector data + { + isTiDB: true, + input: []*model.RowChangedEvent{ + { + StartTs: 418658114257813516, + CommitTs: 418658114257813517, + TableInfo: tableInfoWithVector, + Columns: model.Columns2ColumnDatas([]*model.Column{{ + Name: "a1", + Value: 1, + }, { + Name: "a3", + Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), + }}, tableInfoWithVector), + ApproximateDataSize: 10, + }, + { + StartTs: 418658114257813516, + CommitTs: 418658114257813517, + TableInfo: tableInfoWithVector, + PreColumns: model.Columns2ColumnDatas([]*model.Column{{ + Name: "a1", + Value: 1, + }, { + Name: "a3", + Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), + }}, tableInfoWithVector), + Columns: model.Columns2ColumnDatas([]*model.Column{{ + Name: "a1", + Value: 3, + }, { + Name: "a3", + Value: util.Must(types.ParseVectorFloat32("[1.1,-2,3.33,-4.12,-5]")), + }}, tableInfoWithVector), + ApproximateDataSize: 10, + }, + { + StartTs: 418658114257813516, + CommitTs: 418658114257813517, + TableInfo: tableInfoWithVector, + PreColumns: model.Columns2ColumnDatas([]*model.Column{{ + Name: "a1", + Value: 3, + }, { + Name: "a3", + Value: util.Must(types.ParseVectorFloat32("[1.1,-2,3.33,-4.12,-5]")), + }}, tableInfoWithVector), + ApproximateDataSize: 10, + }, + }, + expected: &preparedDMLs{ + startTs: []model.Ts{418658114257813516}, + sqls: []string{ + "DELETE FROM `common_1`.`uk_without_pk` WHERE (`a1` = ? AND `a3` = ?)", + "UPDATE `common_1`.`uk_without_pk` SET `a1`=CASE WHEN " + + "`a1` = ? AND `a3` = ? THEN ? END, `a3`=CASE WHEN " + + "`a1` = ? AND `a3` = ? THEN ? END WHERE (`a1` = ? AND `a3` = ?)", + "INSERT INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)", + }, + values: [][]interface{}{ + {3, "[1.1,-2,3.33,-4.12,-5]"}, + {1, "[1,2,3,4,5]", 3, 1, "[1,2,3,4,5]", "[1.1,-2,3.33,-4.12,-5]", 1, "[1,2,3,4,5]"}, + {1, "[1,2,3,4,5]"}, + }, + rowCount: 3, + approximateSize: 325, + }, + }, } ctx, cancel := context.WithCancel(context.Background()) From 68a04172d5ec0aa79e95348eab6930ef16118daa Mon Sep 17 00:00:00 2001 From: wk989898 Date: Tue, 3 Sep 2024 14:30:47 +0800 Subject: [PATCH 11/37] update test --- tests/integration_tests/vector/data/data.sql | 18 ++++++++++++++---- tests/integration_tests/vector/run.sh | 3 +-- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/integration_tests/vector/data/data.sql b/tests/integration_tests/vector/data/data.sql index 86371f2d7c5..daa94d721ee 100644 --- a/tests/integration_tests/vector/data/data.sql +++ b/tests/integration_tests/vector/data/data.sql @@ -1,10 +1,10 @@ DROP DATABASE IF EXISTS test; CREATE DATABASE test; use test; -DROP table IF EXISTS test.simple1; -DROP table IF EXISTS test.simple2; +DROP TABLE IF EXISTS test.simple1; +DROP TABLE IF EXISTS test.simple2; -CREATE table test.simple1(id int primary key, data VECTOR(5)); +CREATE TABLE test.simple1(id int primary key, data VECTOR(5)); -- CREATE VECTOR INDEX idx_name1 USING HNSW ON test.simple1(VEC_COSINE_DISTANCE(data)) ; INSERT INTO test.simple1(id, data) VALUES (1, "[1,2,3,4,5]"); INSERT INTO test.simple1(id, data) VALUES (2, '[2,3,4,5,6]'); @@ -12,7 +12,7 @@ INSERT INTO test.simple1(id, data) VALUES (3, '[0.1,0.2,0.3,0.4,0.5]'); INSERT INTO test.simple1(id, data) VALUES (4, '[0,-0.1,-2,2,0.1]'); -CREATE table test.simple2(id int primary key, data VECTOR(5), embedding VECTOR(5) COMMENT "hnsw(distance=cosine)"); +CREATE TABLE test.simple2(id int primary key, data VECTOR(5), embedding VECTOR(5) COMMENT "hnsw(distance=cosine)"); INSERT INTO test.simple2(id, data, embedding) VALUES (1, '[1,2,3,4,5]','[1,2,3,4,5]'); INSERT INTO test.simple2(id, data, embedding) VALUES (2, '[2,3,4,5,6]','[1,2,3,4,5]'); INSERT INTO test.simple2(id, data, embedding) VALUES (3, '[0.1,0.2,0.3,0.4,0.5]','[1,2,3,4,5]'); @@ -22,3 +22,13 @@ DELETE FROM test.simple1 where id=1; DELETE FROM test.simple2 where id=1; DELETE FROM test.simple1 where id=2; DELETE FROM test.simple2 where id=2; + +UPDATE test.simple1 SET data = '[0,-0.1,-2,2.0,0.1]' WHERE id = 3; +UPDATE test.simple2 SET data = '[0,-0.1,-2,2.0,0.1]' WHERE id = 3; + +ALTER TABLE test.simple1 ADD column embedding VECTOR(3) COMMENT "hnsw(distance=cosine)"; +INSERT INTO test.simple1 (id, data, embedding) VALUES (5, '[1,2,3,4,5]', '[1,2,3]'); + +ALTER TABLE test.simple2 DROP column embedding; + +CREATE TABLE test.finish_mark(id int primary key); \ No newline at end of file diff --git a/tests/integration_tests/vector/run.sh b/tests/integration_tests/vector/run.sh index 87a94685fb3..3add2c11cff 100755 --- a/tests/integration_tests/vector/run.sh +++ b/tests/integration_tests/vector/run.sh @@ -32,8 +32,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 3 # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.simple1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists test.simple2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY From 473ea1195e5dd589a90f569922b8e2f9e4b39835 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Wed, 4 Sep 2024 17:36:36 +0800 Subject: [PATCH 12/37] update --- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 13 ++++++++----- cdc/sink/dmlsink/txn/mysql/dml.go | 14 ++++++-------- cdc/sink/dmlsink/txn/mysql/mysql.go | 20 ++++++++------------ pkg/sink/mysql/config.go | 4 ++++ 4 files changed, 26 insertions(+), 25 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 377739a7421..b29ee805a20 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -119,7 +119,7 @@ func NewDDLSink( cfg: cfg, statistics: metrics.NewStatistics(ctx, changefeedID, sink.TxnSink), lastExecutedNormalDDLCache: lruCache, - needFormat: needFormatDDL(db, cfg.IsTiDB), + needFormat: needFormatDDL(db, cfg), } log.Info("MySQL DDL sink is created", @@ -287,16 +287,19 @@ func needSwitchDB(ddl *model.DDLEvent) bool { } // needFormatDDL checks vector type support -func needFormatDDL(db *sql.DB, isTiDB bool) bool { +func needFormatDDL(db *sql.DB, cfg *pmysql.Config) bool { + if !cfg.HasVectorType { + return false + } versionInfo, err := export.SelectVersion(db) if err != nil { - log.Warn("fail to get version", zap.Error(err), zap.Bool("isTiDB", isTiDB)) + log.Warn("fail to get version", zap.Error(err), zap.Bool("isTiDB", cfg.IsTiDB)) return false } serverInfo := version.ParseServerInfo(versionInfo) version := semver.New(defaultSupportVectorVersion) - if !isTiDB || serverInfo.ServerVersion.LessThan(*version) { - log.Error("downstream unsupport vector type. we convert it to longtext", zap.String("supportVersion", version.String()), zap.Bool("isTiDB", isTiDB)) + if !cfg.IsTiDB || serverInfo.ServerVersion.LessThan(*version) { + log.Error("downstream unsupport vector type. we convert it to longtext", zap.String("supportVersion", version.String()), zap.Bool("isTiDB", cfg.IsTiDB)) return true } return false diff --git a/cdc/sink/dmlsink/txn/mysql/dml.go b/cdc/sink/dmlsink/txn/mysql/dml.go index 1bcbcc882ad..d83a860ea9c 100644 --- a/cdc/sink/dmlsink/txn/mysql/dml.go +++ b/cdc/sink/dmlsink/txn/mysql/dml.go @@ -114,16 +114,14 @@ func appendQueryArgs(args []interface{}, col *model.Column) []interface{} { colValBytes, ok := col.Value.([]byte) if ok { args = append(args, string(colValBytes)) - } else { - args = append(args, col.Value) - } - } else { - // convert vector to string - if val, ok := col.Value.(types.VectorFloat32); ok { - col.Value = val.String() + return args } - args = append(args, col.Value) } + switch v := col.Value.(type) { + case types.VectorFloat32: + col.Value = v.String() + } + args = append(args, col.Value) return args } diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index 8cff20985a4..82a2176c4c4 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -339,7 +339,7 @@ func convert2RowChanges( return res } -func convertBinaryToString(cols []*model.ColumnData, tableInfo *model.TableInfo) { +func convertValue(cols []*model.ColumnData, tableInfo *model.TableInfo) { for i, col := range cols { if col == nil { continue @@ -349,8 +349,13 @@ func convertBinaryToString(cols []*model.ColumnData, tableInfo *model.TableInfo) colValBytes, ok := col.Value.([]byte) if ok { cols[i].Value = string(colValBytes) + continue } } + switch v := col.Value.(type) { + case types.VectorFloat32: + cols[i].Value = v.String() + } } } @@ -368,8 +373,8 @@ func (s *mysqlBackend) groupRowsByType( deleteRow := make([]*sqlmodel.RowChange, 0, preAllocateSize) for _, row := range event.Event.Rows { - convertBinaryToString(row.Columns, tableInfo) - convertBinaryToString(row.PreColumns, tableInfo) + convertValue(row.Columns, tableInfo) + convertValue(row.PreColumns, tableInfo) if row.IsInsert() { insertRow = append( @@ -468,15 +473,6 @@ func (s *mysqlBackend) batchSingleTxnDmls( } } - // convert vector to string - for _, value := range values { - for i, val := range value { - switch v := val.(type) { - case types.VectorFloat32: - value[i] = v.String() - } - } - } return } diff --git a/pkg/sink/mysql/config.go b/pkg/sink/mysql/config.go index 4adf68639a8..841d385b298 100644 --- a/pkg/sink/mysql/config.go +++ b/pkg/sink/mysql/config.go @@ -77,6 +77,8 @@ const ( // defaultcachePrepStmts is the default value of cachePrepStmts defaultCachePrepStmts = true + + defaultHasVectorType = true ) type urlConfig struct { @@ -117,6 +119,7 @@ type Config struct { // IsBDRModeSupported is true if the downstream is TiDB and write source is existed. // write source exists when the downstream is TiDB and version is greater than or equal to v6.5.0. IsWriteSourceExisted bool + HasVectorType bool // HasVectorType is true if the data is vector SourceID uint64 BatchDMLEnable bool @@ -140,6 +143,7 @@ func NewConfig() *Config { MultiStmtEnable: defaultMultiStmtEnable, CachePrepStmts: defaultCachePrepStmts, SourceID: config.DefaultTiDBSourceID, + HasVectorType: defaultHasVectorType, } } From 053633232faa8d09538bfc5d35537e6ab9b00635 Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Mon, 2 Sep 2024 18:18:19 +0800 Subject: [PATCH 13/37] sink(ticdc): use admin statement to query async ddl status (#11535) close pingcap/tiflow#11521 --- cdc/sink/ddlsink/mysql/async_ddl.go | 78 ++++++++++++++++-------- cdc/sink/ddlsink/mysql/async_ddl_test.go | 28 ++++++++- 2 files changed, 79 insertions(+), 27 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/async_ddl.go b/cdc/sink/ddlsink/mysql/async_ddl.go index 1692a394f37..8e480bb06b0 100644 --- a/cdc/sink/ddlsink/mysql/async_ddl.go +++ b/cdc/sink/ddlsink/mysql/async_ddl.go @@ -15,25 +15,29 @@ package mysql import ( "context" - "database/sql" "fmt" "time" "github.com/pingcap/log" + "github.com/pingcap/tidb/dumpling/export" timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) +const timeout = 5 * time.Second + +// TODO: Use the flollowing SQL to check the ddl job status after tidb optimize +// the information_schema.ddl_jobs table. Ref: https://github.com/pingcap/tidb/issues/55725 +// +// SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY +// FROM information_schema.ddl_jobs var checkRunningAddIndexSQL = ` -SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY -FROM information_schema.ddl_jobs +ADMIN SHOW DDL JOBS 1 WHERE DB_NAME = "%s" AND TABLE_NAME = "%s" AND JOB_TYPE LIKE "add index%%" - AND (STATE = "running" OR STATE = "queueing") -LIMIT 1; + AND (STATE = "running" OR STATE = "queueing"); ` func (m *DDLSink) shouldAsyncExecDDL(ddl *model.DDLEvent) bool { @@ -92,9 +96,23 @@ func (m *DDLSink) asyncExecDDL(ctx context.Context, ddl *model.DDLEvent) error { } } +func (m *DDLSink) needWaitAsyncExecDone(t timodel.ActionType) bool { + if !m.cfg.IsTiDB { + return false + } + switch t { + case timodel.ActionCreateTable, timodel.ActionCreateTables: + return false + case timodel.ActionCreateSchema: + return false + default: + return true + } +} + // Should always wait for async ddl done before executing the next ddl. func (m *DDLSink) waitAsynExecDone(ctx context.Context, ddl *model.DDLEvent) { - if !m.cfg.IsTiDB { + if !m.needWaitAsyncExecDone(ddl.Type) { return } @@ -105,9 +123,6 @@ func (m *DDLSink) waitAsynExecDone(ctx context.Context, ddl *model.DDLEvent) { if ddl.PreTableInfo != nil { tables[ddl.PreTableInfo.TableName] = struct{}{} } - if len(tables) == 0 || m.checkAsyncExecDDLDone(ctx, tables) { - return - } log.Debug("wait async exec ddl done", zap.String("namespace", m.id.Namespace), @@ -115,6 +130,10 @@ func (m *DDLSink) waitAsynExecDone(ctx context.Context, ddl *model.DDLEvent) { zap.Any("tables", tables), zap.Uint64("commitTs", ddl.CommitTs), zap.String("ddl", ddl.Query)) + if len(tables) == 0 || m.checkAsyncExecDDLDone(ctx, tables) { + return + } + ticker := time.NewTicker(5 * time.Second) defer ticker.Stop() for { @@ -131,6 +150,8 @@ func (m *DDLSink) waitAsynExecDone(ctx context.Context, ddl *model.DDLEvent) { } func (m *DDLSink) checkAsyncExecDDLDone(ctx context.Context, tables map[model.TableName]struct{}) bool { + ctx, cancel := context.WithTimeout(ctx, timeout) + defer cancel() for table := range tables { done := m.doCheck(ctx, table) if !done { @@ -141,6 +162,7 @@ func (m *DDLSink) checkAsyncExecDDLDone(ctx context.Context, tables map[model.Ta } func (m *DDLSink) doCheck(ctx context.Context, table model.TableName) (done bool) { + start := time.Now() if v, ok := m.lastExecutedNormalDDLCache.Get(table); ok { ddlType := v.(timodel.ActionType) if ddlType == timodel.ActionAddIndex { @@ -152,35 +174,41 @@ func (m *DDLSink) doCheck(ctx context.Context, table model.TableName) (done bool return true } - ret := m.db.QueryRowContext(ctx, fmt.Sprintf(checkRunningAddIndexSQL, table.Schema, table.Table)) - if ret.Err() != nil { + rows, err := m.db.QueryContext(ctx, fmt.Sprintf(checkRunningAddIndexSQL, table.Schema, table.Table)) + defer func() { + if rows != nil { + _ = rows.Err() + } + }() + if err != nil { log.Error("check async exec ddl failed", zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID), - zap.Error(ret.Err())) + zap.Error(err)) return true } - var jobID, jobType, schemaState, schemaID, tableID, state, query string - if err := ret.Scan(&jobID, &jobType, &schemaState, &schemaID, &tableID, &state, &query); err != nil { - if !errors.Is(err, sql.ErrNoRows) { - log.Error("check async exec ddl failed", - zap.String("namespace", m.id.Namespace), - zap.String("changefeed", m.id.ID), - zap.Error(err)) - } + rets, err := export.GetSpecifiedColumnValuesAndClose(rows, "JOB_ID", "JOB_TYPE", "SCHEMA_STATE", "STATE") + if err != nil { + log.Error("check async exec ddl failed", + zap.String("namespace", m.id.Namespace), + zap.String("changefeed", m.id.ID), + zap.Error(err)) return true } + if len(rets) == 0 { + return true + } + ret := rets[0] + jobID, jobType, schemaState, state := ret[0], ret[1], ret[2], ret[3] log.Info("async ddl is still running", zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID), + zap.Duration("checkDuration", time.Since(start)), zap.String("table", table.String()), zap.String("jobID", jobID), zap.String("jobType", jobType), zap.String("schemaState", schemaState), - zap.String("schemaID", schemaID), - zap.String("tableID", tableID), - zap.String("state", state), - zap.String("query", query)) + zap.String("state", state)) return false } diff --git a/cdc/sink/ddlsink/mysql/async_ddl_test.go b/cdc/sink/ddlsink/mysql/async_ddl_test.go index 871080c2eb5..b05f09bb979 100644 --- a/cdc/sink/ddlsink/mysql/async_ddl_test.go +++ b/cdc/sink/ddlsink/mysql/async_ddl_test.go @@ -45,8 +45,14 @@ func TestWaitAsynExecDone(t *testing.T) { // Case 1: there is a running add index job mock.ExpectQuery(fmt.Sprintf(checkRunningAddIndexSQL, "test", "sbtest0")).WillReturnRows( - sqlmock.NewRows([]string{"JOB_ID", "JOB_TYPE", "SCHEMA_STATE", "SCHEMA_ID", "TABLE_ID", "STATE", "QUERY"}). - AddRow("1", "add index", "running", "1", "1", "running", "Create index idx1 on test.sbtest0(a)"), + sqlmock.NewRows([]string{ + "JOB_ID", "DB_NAME", "TABLE_NAME", "JOB_TYPE", "SCHEMA_STATE", "SCHEMA_ID", "TABLE_ID", + "ROW_COUNT", "CREATE_TIME", "START_TIME", "END_TIME", "STATE", + }).AddRow( + 1, "test", "sbtest0", "add index", "write reorganization", 1, 1, 0, time.Now(), nil, time.Now(), "running", + ).AddRow( + 2, "test", "sbtest0", "add index", "write reorganization", 1, 1, 0, time.Now(), time.Now(), time.Now(), "queueing", + ), ) // Case 2: there is no running add index job // Case 3: no permission to query ddl_jobs, TiDB will return empty result @@ -157,3 +163,21 @@ func TestAsyncExecAddIndex(t *testing.T) { require.True(t, time.Since(start) >= 10*time.Second) sink.Close() } + +func TestNeedWaitAsyncExecDone(t *testing.T) { + sink := &DDLSink{ + cfg: &pmysql.Config{ + IsTiDB: false, + }, + } + require.False(t, sink.needWaitAsyncExecDone(timodel.ActionTruncateTable)) + + sink.cfg.IsTiDB = true + require.True(t, sink.needWaitAsyncExecDone(timodel.ActionTruncateTable)) + require.True(t, sink.needWaitAsyncExecDone(timodel.ActionDropTable)) + require.True(t, sink.needWaitAsyncExecDone(timodel.ActionDropIndex)) + + require.False(t, sink.needWaitAsyncExecDone(timodel.ActionCreateTable)) + require.False(t, sink.needWaitAsyncExecDone(timodel.ActionCreateTables)) + require.False(t, sink.needWaitAsyncExecDone(timodel.ActionCreateSchema)) +} From 17dd1628813f733a75625291463dc91b38fd3160 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 5 Sep 2024 18:51:48 +0800 Subject: [PATCH 14/37] update --- cdc/sink/util/vector.go | 48 +++++++++++++++++++++++++++++ pkg/sink/codec/canal/canal_entry.go | 2 +- pkg/sink/codec/debezium/codec.go | 12 ++++++++ 3 files changed, 61 insertions(+), 1 deletion(-) diff --git a/cdc/sink/util/vector.go b/cdc/sink/util/vector.go index 8747c05cebf..1138a64517e 100644 --- a/cdc/sink/util/vector.go +++ b/cdc/sink/util/vector.go @@ -13,6 +13,17 @@ package util +import ( + "encoding/json" + "log" + "reflect" + "strconv" + "unsafe" + + "github.com/pingcap/tidb/pkg/types" + "go.uber.org/zap" +) + // CheckVectorIndexForColumnModify checks Vector Index constraints for MODIFY COLUMN. // func CheckVectorIndexForColumnModify(oldCol *table.Column, newCol *table.Column) error { // if oldCol.VectorIndex == nil && newCol.VectorIndex == nil { @@ -35,3 +46,40 @@ package util // } // return nil // } + +func ParseVectorFromElement(values []float32) (types.VectorFloat32, error) { + dim := len(values) + if err := types.CheckVectorDimValid(dim); err != nil { + return types.ZeroVectorFloat32, err + } + vec := types.InitVectorFloat32(dim) + copy(vec.Elements(), values) + return vec, nil +} + +func VectorElement2String(elements []interface{}) string { + buf := make([]byte, 0, 2+len(elements)*2) + buf = append(buf, '[') + for i, val := range elements { + if i > 0 { + buf = append(buf, ',') + } + switch v := val.(type) { + case json.Number: + num, err := v.Float64() + if err != nil { + log.Panic("failed to decode val", zap.Any("val", val), zap.Error(err)) + } + buf = strconv.AppendFloat(buf, num, 'f', -1, 32) + case float32: + buf = strconv.AppendFloat(buf, float64(v), 'f', -1, 32) + case float64: + buf = strconv.AppendFloat(buf, v, 'f', -1, 32) + default: + log.Panic("failed to decode val type", zap.Any("val", val), zap.Any("type", reflect.TypeOf(v))) + } + } + buf = append(buf, ']') + // buf is not used elsewhere, so it's safe to just cast to String + return unsafe.String(unsafe.SliceData(buf), len(buf)) +} diff --git a/pkg/sink/codec/canal/canal_entry.go b/pkg/sink/codec/canal/canal_entry.go index a79ff0f5fd3..ac58f8d1d7e 100644 --- a/pkg/sink/codec/canal/canal_entry.go +++ b/pkg/sink/codec/canal/canal_entry.go @@ -111,7 +111,7 @@ func (b *canalEntryBuilder) formatValue(value interface{}, isBinary bool) (resul result = string(v) } case types.VectorFloat32: - result = value.(types.VectorFloat32).String() + result = v.String() default: result = fmt.Sprintf("%v", v) } diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index 22a9bca7940..75c3ada818a 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -240,6 +240,13 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("field", col.Name) }) + case mysql.TypeTiDBVectorFloat32: + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) + writer.WriteStringField("name", "io.debezium.data.TiDBVectorFloat32") + writer.WriteStringField("field", col.Name) + }) default: log.Warn( "meet unsupported field type", @@ -502,6 +509,11 @@ func (c *dbzCodec) writeDebeziumFieldValue( return nil } + case mysql.TypeTiDBVectorFloat32: + v := col.Value.(types.VectorFloat32).String() + writer.WriteStringField(col.Name, v) + return nil + // Note: Although Debezium's doc claims to use INT32 for INT, but it // actually uses INT64. Debezium also uses INT32 for SMALLINT. // So we only handle with TypeLonglong here. From 3d41f033f2c3d68b9faca7dccb55ed09c0374434 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 6 Sep 2024 13:14:18 +0800 Subject: [PATCH 15/37] chore --- pkg/sink/codec/simple/message.go | 4 ++++ tests/integration_tests/vector/run.sh | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/pkg/sink/codec/simple/message.go b/pkg/sink/codec/simple/message.go index fe7ab25125b..0453dcf8fa3 100644 --- a/pkg/sink/codec/simple/message.go +++ b/pkg/sink/codec/simple/message.go @@ -638,6 +638,8 @@ func (a *avroMarshaller) encodeValue4Avro( return v, "double" case string: return v, "string" + case tiTypes.VectorFloat32: + return v.String(), "string" default: log.Panic("unexpected type for avro value", zap.Any("value", value)) } @@ -714,6 +716,8 @@ func encodeValue( } else { result = string(v) } + case tiTypes.VectorFloat32: + result = v.String() default: result = fmt.Sprintf("%v", v) } diff --git a/tests/integration_tests/vector/run.sh b/tests/integration_tests/vector/run.sh index 3add2c11cff..3976b5be60e 100755 --- a/tests/integration_tests/vector/run.sh +++ b/tests/integration_tests/vector/run.sh @@ -20,13 +20,13 @@ function run() { TOPIC_NAME="vector-$RANDOM" case $SINK_TYPE in - kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; + kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&max-message-bytes=10485760" ;; storage) SINK_URI="file://$WORK_DIR/storage_test/$TOPIC_NAME?protocol=canal-json&enable-tidb-extension=true" ;; *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/" ;; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" case $SINK_TYPE in - kafka) run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; + kafka) run_kafka_consumer $WORK_DIR $SINK_URI;; storage) run_storage_consumer $WORK_DIR $SINK_URI "" "" ;; esac run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} From 3402020b5a1941e36548c55d0b80c028a691b939 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 6 Sep 2024 13:19:00 +0800 Subject: [PATCH 16/37] update --- cdc/sink/dmlsink/txn/mysql/mysql.go | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index 82a2176c4c4..fa2de7e27fb 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -345,16 +345,13 @@ func convertValue(cols []*model.ColumnData, tableInfo *model.TableInfo) { continue } colInfo := tableInfo.ForceGetColumnInfo(col.ColumnID) - if colInfo.GetCharset() != "" && colInfo.GetCharset() != charset.CharsetBin { - colValBytes, ok := col.Value.([]byte) - if ok { - cols[i].Value = string(colValBytes) - continue - } - } switch v := col.Value.(type) { case types.VectorFloat32: cols[i].Value = v.String() + case []byte: + if colInfo.GetCharset() != "" && colInfo.GetCharset() != charset.CharsetBin { + cols[i].Value = string(v) + } } } } From fd8d907079a004b150cfb8315cd62ace5535ace6 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 6 Sep 2024 13:24:05 +0800 Subject: [PATCH 17/37] . --- cdc/sink/dmlsink/txn/mysql/dml.go | 10 ++++------ cdc/sink/dmlsink/txn/mysql/mysql.go | 4 ++-- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/cdc/sink/dmlsink/txn/mysql/dml.go b/cdc/sink/dmlsink/txn/mysql/dml.go index d83a860ea9c..a82d7fc65c4 100644 --- a/cdc/sink/dmlsink/txn/mysql/dml.go +++ b/cdc/sink/dmlsink/txn/mysql/dml.go @@ -110,14 +110,12 @@ func prepareReplace( // will automatically set `_binary` charset for that column, which is not expected. // See https://github.com/go-sql-driver/mysql/blob/ce134bfc/connection.go#L267 func appendQueryArgs(args []interface{}, col *model.Column) []interface{} { - if col.Charset != "" && col.Charset != charset.CharsetBin { - colValBytes, ok := col.Value.([]byte) - if ok { - args = append(args, string(colValBytes)) + switch v := col.Value.(type) { + case []byte: + if col.Charset != "" && col.Charset != charset.CharsetBin { + args = append(args, string(v)) return args } - } - switch v := col.Value.(type) { case types.VectorFloat32: col.Value = v.String() } diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index fa2de7e27fb..649219d6815 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -346,12 +346,12 @@ func convertValue(cols []*model.ColumnData, tableInfo *model.TableInfo) { } colInfo := tableInfo.ForceGetColumnInfo(col.ColumnID) switch v := col.Value.(type) { - case types.VectorFloat32: - cols[i].Value = v.String() case []byte: if colInfo.GetCharset() != "" && colInfo.GetCharset() != charset.CharsetBin { cols[i].Value = string(v) } + case types.VectorFloat32: + cols[i].Value = v.String() } } } From 4a5d3011284454a3523aa4676c1834a772556b7a Mon Sep 17 00:00:00 2001 From: nhsmw Date: Thu, 5 Sep 2024 11:00:05 +0800 Subject: [PATCH 18/37] test(ticdc,dm): modify download-integration-test-binaries script (#11564) close pingcap/tiflow#11563 --- .../download-integration-test-binaries.sh | 19 +++++++++++--- scripts/download-integration-test-binaries.sh | 25 +++++++++++++++---- 2 files changed, 36 insertions(+), 8 deletions(-) diff --git a/dm/tests/download-integration-test-binaries.sh b/dm/tests/download-integration-test-binaries.sh index 21a505664d9..34dab604e85 100755 --- a/dm/tests/download-integration-test-binaries.sh +++ b/dm/tests/download-integration-test-binaries.sh @@ -41,6 +41,19 @@ function download() { wget --no-verbose --retry-connrefused --waitretry=1 -t 3 -O "${file_path}" "${url}" } +function get_sha1() { + local repo="$1" + local branch="$2" + file_server_url="http://fileserver.pingcap.net" + sha1=$(curl -s "${file_server_url}/download/refs/pingcap/${repo}/${branch}/sha1") + if [ $? -ne 0 ] || echo "$sha1" | grep -q "Error"; then + echo "Failed to get sha1 with repo ${repo} branch ${branch}: $sha1. use branch master to instead" >&2 + branch=master + sha1=$(curl -s "${file_server_url}/download/refs/pingcap/${repo}/${branch}/sha1") + fi + echo $sha1 +} + # Specify the download branch. branch=$1 @@ -48,9 +61,9 @@ branch=$1 file_server_url="http://fileserver.pingcap.net" # Get sha1 based on branch name. -tidb_sha1=$(curl "${file_server_url}/download/refs/pingcap/tidb/${branch}/sha1") -tikv_sha1=$(curl "${file_server_url}/download/refs/pingcap/tikv/${branch}/sha1") -pd_sha1=$(curl "${file_server_url}/download/refs/pingcap/pd/${branch}/sha1") +tidb_sha1=$(get_sha1 "tidb" "$branch") +tikv_sha1=$(get_sha1 "tikv" "$branch") +pd_sha1=$(get_sha1 "pd" "$branch") tidb_tools_sha1=$(curl "${file_server_url}/download/refs/pingcap/tidb-tools/master/sha1") # All download links. diff --git a/scripts/download-integration-test-binaries.sh b/scripts/download-integration-test-binaries.sh index d5f0bba5984..60731df7ee6 100755 --- a/scripts/download-integration-test-binaries.sh +++ b/scripts/download-integration-test-binaries.sh @@ -108,22 +108,37 @@ function download_community_binaries() { chmod a+x third_bin/* } +function get_sha1() { + local repo="$1" + local branch="$2" + file_server_url="http://fileserver.pingcap.net" + sha1=$(curl -s "${file_server_url}/download/refs/pingcap/${repo}/${branch}/sha1") + if [ $? -ne 0 ] || echo "$sha1" | grep -q "Error"; then + echo "Failed to get sha1 with repo ${repo} branch ${branch}: $sha1. use branch master to instead" >&2 + branch=master + sha1=$(curl -s "${file_server_url}/download/refs/pingcap/${repo}/${branch}/sha1") + fi + echo "$branch:$sha1" +} + function download_binaries() { color-green "Download binaries..." # PingCAP file server URL. file_server_url="http://fileserver.pingcap.net" # Get sha1 based on branch name. - tidb_sha1=$(curl "${file_server_url}/download/refs/pingcap/tidb/${branch}/sha1") - tikv_sha1=$(curl "${file_server_url}/download/refs/pingcap/tikv/${branch}/sha1") - pd_sha1=$(curl "${file_server_url}/download/refs/pingcap/pd/${branch}/sha1") - tiflash_sha1=$(curl "${file_server_url}/download/refs/pingcap/tiflash/${branch}/sha1") + tidb_sha1=$(get_sha1 "tidb" "$branch" | cut -d':' -f2) + tikv_sha1=$(get_sha1 "tikv" "$branch" | cut -d':' -f2) + pd_sha1=$(get_sha1 "pd" "$branch" | cut -d':' -f2) + tiflash_branch_sha1=$(get_sha1 "tiflash" "$branch") + tiflash_branch=$(echo "$tiflash_branch_sha1" | cut -d':' -f1) + tiflash_sha1=$(echo "$tiflash_branch_sha1" | cut -d':' -f2) # All download links. tidb_download_url="${file_server_url}/download/builds/pingcap/tidb/${tidb_sha1}/centos7/tidb-server.tar.gz" tikv_download_url="${file_server_url}/download/builds/pingcap/tikv/${tikv_sha1}/centos7/tikv-server.tar.gz" pd_download_url="${file_server_url}/download/builds/pingcap/pd/${pd_sha1}/centos7/pd-server.tar.gz" - tiflash_download_url="${file_server_url}/download/builds/pingcap/tiflash/${branch}/${tiflash_sha1}/centos7/tiflash.tar.gz" + tiflash_download_url="${file_server_url}/download/builds/pingcap/tiflash/${tiflash_branch}/${tiflash_sha1}/centos7/tiflash.tar.gz" minio_download_url="${file_server_url}/download/minio.tar.gz" go_ycsb_download_url="${file_server_url}/download/builds/pingcap/go-ycsb/test-br/go-ycsb" etcd_download_url="${file_server_url}/download/builds/pingcap/cdc/etcd-v3.4.7-linux-amd64.tar.gz" From 059bd0f56ff65d9f17511e6bdbc824ee27e87d5a Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 6 Sep 2024 14:09:04 +0800 Subject: [PATCH 19/37] update --- cdc/sink/dmlsink/txn/mysql/mysql.go | 2 +- dm/tests/download-compatibility-test-binaries.sh | 15 ++++++++++++++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index 649219d6815..60fd558f424 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -344,9 +344,9 @@ func convertValue(cols []*model.ColumnData, tableInfo *model.TableInfo) { if col == nil { continue } - colInfo := tableInfo.ForceGetColumnInfo(col.ColumnID) switch v := col.Value.(type) { case []byte: + colInfo := tableInfo.ForceGetColumnInfo(col.ColumnID) if colInfo.GetCharset() != "" && colInfo.GetCharset() != charset.CharsetBin { cols[i].Value = string(v) } diff --git a/dm/tests/download-compatibility-test-binaries.sh b/dm/tests/download-compatibility-test-binaries.sh index cceb8c4432d..c4797084447 100755 --- a/dm/tests/download-compatibility-test-binaries.sh +++ b/dm/tests/download-compatibility-test-binaries.sh @@ -41,6 +41,19 @@ function download() { wget --no-verbose --retry-connrefused --waitretry=1 -t 3 -O "${file_path}" "${url}" } +function get_sha1() { + local repo="$1" + local branch="$2" + file_server_url="http://fileserver.pingcap.net" + sha1=$(curl -s "${file_server_url}/download/refs/pingcap/${repo}/${branch}/sha1") + if [ $? -ne 0 ] || echo "$sha1" | grep -q "Error"; then + echo "Failed to get sha1 with repo ${repo} branch ${branch}: $sha1. use branch master to instead" >&2 + branch=master + sha1=$(curl -s "${file_server_url}/download/refs/pingcap/${repo}/${branch}/sha1") + fi + echo $sha1 +} + # Specify the download branch. branch=$1 @@ -48,7 +61,7 @@ branch=$1 file_server_url="http://fileserver.pingcap.net" # Get sha1 based on branch name. -tidb_sha1=$(curl "${file_server_url}/download/refs/pingcap/tidb/${branch}/sha1") +tidb_sha1=$(get_sha1 "tidb" "$branch") # All download links. tidb_download_url="${file_server_url}/download/builds/pingcap/tidb/${tidb_sha1}/centos7/tidb-server.tar.gz" From ed6cf61d5622708d752c9d3cbf3e0d0584ec4534 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 6 Sep 2024 14:25:39 +0800 Subject: [PATCH 20/37] add test --- tests/integration_tests/run_group.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index 991ba3d188a..df2ab20ed4d 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -42,7 +42,7 @@ groups=( # G04 'foreign_key ddl_puller_lag ddl_only_block_related_table changefeed_auto_stop' # G05 - 'charset_gbk ddl_manager multi_source' + 'charset_gbk ddl_manager multi_source vector' # G06 'sink_retry changefeed_error ddl_sequence resourcecontrol' # G07 pulsar oauth2 authentication enabled From 8ab796cbc9b275205170aaf997d650c155de387b Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 6 Sep 2024 17:46:34 +0800 Subject: [PATCH 21/37] fix test --- cdc/entry/mounter.go | 2 + cdc/entry/mounter_test.go | 11 +++- cdc/entry/schema_storage_test.go | 8 --- cdc/entry/schema_test_helper.go | 2 - cdc/sink/dmlsink/txn/mysql/mysql_test.go | 6 +-- dm/pkg/conn/mockdb.go | 1 - pkg/election/storage_orm_test.go | 24 ++++----- pkg/filter/filter_test_helper.go | 2 - pkg/sink/codec/avro/avro.go | 3 +- pkg/sink/codec/canal/canal_entry_test.go | 8 +-- .../canal_json_row_event_encoder_test.go | 29 ++++++++-- pkg/sink/codec/common/verify_checksum.go | 4 ++ .../codec/open/open_protocol_encoder_test.go | 8 ++- pkg/sink/codec/simple/encoder_test.go | 53 ++++++++++++++++--- pkg/sink/codec/simple/message.go | 25 ++++++--- pkg/sink/codec/utils/test_utils.go | 3 ++ tests/integration_tests/vector/run.sh | 2 +- 17 files changed, 135 insertions(+), 56 deletions(-) diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 7eec2a8bd9d..c2ea2319ad2 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -592,6 +592,8 @@ func newDatum(value interface{}, ft types.FieldType) (types.Datum, error) { return types.NewFloat32Datum(value.(float32)), nil case mysql.TypeDouble: return types.NewFloat64Datum(value.(float64)), nil + case mysql.TypeTiDBVectorFloat32: + return types.NewVectorFloat32Datum(value.(types.VectorFloat32)), nil default: log.Panic("unexpected mysql type found", zap.Any("type", ft.GetType())) } diff --git a/cdc/entry/mounter_test.go b/cdc/entry/mounter_test.go index 739080d63b4..680b656fbe1 100644 --- a/cdc/entry/mounter_test.go +++ b/cdc/entry/mounter_test.go @@ -269,8 +269,6 @@ func testMounterDisableOldValue(t *testing.T, tc struct { ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { // we can update the tidb config here }) - session.SetSchemaLease(0) - session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() @@ -1682,4 +1680,13 @@ func TestFormatColVal(t *testing.T) { require.NoError(t, err) require.Equal(t, float32(0), value) require.NotZero(t, warn) + + vector, _ := types.ParseVectorFloat32("[1,2,3,4,5]") + ftTypeVector := types.NewFieldType(mysql.TypeTiDBVectorFloat32) + col = &timodel.ColumnInfo{FieldType: *ftTypeVector} + datum.SetVectorFloat32(vector) + value, _, warn, err = formatColVal(datum, col) + require.NoError(t, err) + require.Equal(t, vector, value) + require.Zero(t, warn) } diff --git a/cdc/entry/schema_storage_test.go b/cdc/entry/schema_storage_test.go index 4ae58289754..6128a557810 100644 --- a/cdc/entry/schema_storage_test.go +++ b/cdc/entry/schema_storage_test.go @@ -673,8 +673,6 @@ func TestCreateSnapFromMeta(t *testing.T) { require.Nil(t, err) defer store.Close() //nolint:errcheck - session.SetSchemaLease(0) - session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() @@ -708,8 +706,6 @@ func TestExplicitTables(t *testing.T) { require.Nil(t, err) defer store.Close() //nolint:errcheck - session.SetSchemaLease(0) - session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() @@ -859,8 +855,6 @@ func TestSchemaStorage(t *testing.T) { ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { conf.AlterPrimaryKey = true }) - session.SetSchemaLease(0) - session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() @@ -952,8 +946,6 @@ func TestHandleKey(t *testing.T) { require.Nil(t, err) defer store.Close() //nolint:errcheck - session.SetSchemaLease(0) - session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() diff --git a/cdc/entry/schema_test_helper.go b/cdc/entry/schema_test_helper.go index fba513ba08c..324fc2bfbd7 100644 --- a/cdc/entry/schema_test_helper.go +++ b/cdc/entry/schema_test_helper.go @@ -61,8 +61,6 @@ func NewSchemaTestHelperWithReplicaConfig( ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { conf.AlterPrimaryKey = true }) - session.SetSchemaLease(0) - session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.NoError(t, err) domain.SetStatsUpdating(true) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index fe853e408ce..7edcc7baa19 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -126,7 +126,7 @@ func TestPrepareDML(t *testing.T) { Name: "a3", Type: mysql.TypeTiDBVectorFloat32, }, - }, [][]int{{}}) + }, [][]int{{1, 2}}) testCases := []struct { input []*model.RowChangedEvent @@ -220,10 +220,10 @@ func TestPrepareDML(t *testing.T) { }, expected: &preparedDMLs{ startTs: []model.Ts{418658114257813518}, - sqls: []string{"REPLACE INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)"}, + sqls: []string{"INSERT INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)"}, values: [][]interface{}{{"[1,2,3,4,5]", "[1.1,-2,3.33,-4.12,-5]"}}, rowCount: 1, - approximateSize: 64, + approximateSize: 63, }, }, } diff --git a/dm/pkg/conn/mockdb.go b/dm/pkg/conn/mockdb.go index f9d943071b5..700f4d3e6b9 100644 --- a/dm/pkg/conn/mockdb.go +++ b/dm/pkg/conn/mockdb.go @@ -147,7 +147,6 @@ func NewCluster() (*Cluster, error) { } cluster.Storage = storage - session.SetSchemaLease(0) session.DisableStats4Test() dom, err := session.BootstrapSession(storage) if err != nil { diff --git a/pkg/election/storage_orm_test.go b/pkg/election/storage_orm_test.go index 443610f5e92..ae75a1fbe27 100644 --- a/pkg/election/storage_orm_test.go +++ b/pkg/election/storage_orm_test.go @@ -34,9 +34,9 @@ func newORMStorageAndMock(t *testing.T) (*ORMStorage, sqlmock.Sqlmock) { db, err := ormUtil.NewGormDB(backendDB, "mysql") require.NoError(t, err) mock.ExpectQuery("SELECT SCHEMA_NAME from Information_schema.SCHEMATA " + - "where SCHEMA_NAME LIKE ? ORDER BY SCHEMA_NAME=? DESC limit 1").WillReturnRows( + "where SCHEMA_NAME LIKE ? ORDER BY SCHEMA_NAME=? DESC,SCHEMA_NAME limit 1").WillReturnRows( sqlmock.NewRows([]string{"SCHEMA_NAME"})) - mock.ExpectExec("CREATE TABLE `test` (`id` int(10) unsigned,`leader_id` text NOT NULL," + + mock.ExpectExec("CREATE TABLE `test` (`id` int(10) unsigned AUTO_INCREMENT,`leader_id` text NOT NULL," + "`record` text,`version` bigint(20) unsigned NOT NULL,PRIMARY KEY (`id`))"). WillReturnResult(sqlmock.NewResult(0, 0)) @@ -49,8 +49,8 @@ func newORMStorageAndMock(t *testing.T) (*ORMStorage, sqlmock.Sqlmock) { func TestORMStorageGetEmptyRecord(t *testing.T) { s, mock := newORMStorageAndMock(t) - mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT 1"). - WithArgs(1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"})) + mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT ?"). + WithArgs(1, 1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"})) record, err := s.Get(context.Background()) require.NoError(t, err) require.Equal(t, &Record{}, record) @@ -76,8 +76,8 @@ func TestORMStorageGetExistingRecord(t *testing.T) { recordBytes, err := json.Marshal(expectedRecord) require.NoError(t, err) - mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT 1"). - WithArgs(1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"}). + mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT ?"). + WithArgs(1, 1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"}). AddRow(1, "id1", recordBytes, 1)) record, err := s.Get(context.Background()) require.NoError(t, err) @@ -189,8 +189,8 @@ func TestORMStorageTxnWithLeaderCheck(t *testing.T) { s, mock := newORMStorageAndMock(t) mock.ExpectBegin() - mock.ExpectQuery("SELECT `leader_id` FROM `test` WHERE id = ? and leader_id = ? LIMIT 1 LOCK IN SHARE MODE"). - WithArgs(leaderRowID, "leader1").WillReturnRows(sqlmock.NewRows([]string{"leader_id"})) + mock.ExpectQuery("SELECT `leader_id` FROM `test` WHERE id = ? and leader_id = ? LIMIT ? LOCK IN SHARE MODE"). + WithArgs(leaderRowID, "leader1", 1).WillReturnRows(sqlmock.NewRows([]string{"leader_id"})) mock.ExpectRollback() doNothing := func(*gorm.DB) error { return nil @@ -199,11 +199,11 @@ func TestORMStorageTxnWithLeaderCheck(t *testing.T) { require.ErrorIs(t, err, errors.ErrElectorNotLeader) mock.ExpectBegin() - mock.ExpectQuery("SELECT `leader_id` FROM `test` WHERE id = ? and leader_id = ? LIMIT 1 LOCK IN SHARE MODE"). - WithArgs(leaderRowID, "leader1"). + mock.ExpectQuery("SELECT `leader_id` FROM `test` WHERE id = ? and leader_id = ? LIMIT ? LOCK IN SHARE MODE"). + WithArgs(leaderRowID, "leader1", 1). WillReturnRows(sqlmock.NewRows([]string{"leader_id"}).AddRow("leader1")) - mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT 1"). - WithArgs(1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"})) + mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT ?"). + WithArgs(1, 1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"})) mock.ExpectCommit() doTxn := func(tx *gorm.DB) error { _, err := s.Get(context.Background()) diff --git a/pkg/filter/filter_test_helper.go b/pkg/filter/filter_test_helper.go index e5b4afedb23..a30915f5c38 100644 --- a/pkg/filter/filter_test_helper.go +++ b/pkg/filter/filter_test_helper.go @@ -46,8 +46,6 @@ func newTestHelper(t *testing.T) *testHelper { ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { conf.AlterPrimaryKey = true }) - session.SetSchemaLease(0) - session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) domain.SetStatsUpdating(true) diff --git a/pkg/sink/codec/avro/avro.go b/pkg/sink/codec/avro/avro.go index 2b22d68689b..368f4030471 100644 --- a/pkg/sink/codec/avro/avro.go +++ b/pkg/sink/codec/avro/avro.go @@ -817,8 +817,7 @@ func (a *BatchEncoder) columnToAvroSchema( }, nil case mysql.TypeTiDBVectorFloat32: return avroSchema{ - Type: "string", - // Type: "array", + Type: "string", Parameters: map[string]string{tidbType: tt}, }, nil default: diff --git a/pkg/sink/codec/canal/canal_entry_test.go b/pkg/sink/codec/canal/canal_entry_test.go index fa4179f1a42..43ef22bec0c 100644 --- a/pkg/sink/codec/canal/canal_entry_test.go +++ b/pkg/sink/codec/canal/canal_entry_test.go @@ -35,8 +35,8 @@ func TestInsert(t *testing.T) { name varchar(32), tiny tinyint, comment text, - bb blob - vec vector)` + bb blob, + vec vector(5))` _ = helper.DDL2Event(sql) event := helper.DML2Event(`insert into test.t values(1, "Bob", 127, "测试", "测试blob", '[1,2,3,4,5]')`, "test", "t") @@ -98,13 +98,13 @@ func TestInsert(t *testing.T) { require.NoError(t, err) require.Equal(t, "测试blob", s) require.Equal(t, "blob", col.GetMysqlType()) - case "vectorfloat32": + case "vec": require.Equal(t, int32(internal.JavaSQLTypeVARCHAR), col.GetSqlType()) require.False(t, col.GetIsKey()) require.False(t, col.GetIsNull()) require.NoError(t, err) require.Equal(t, "[1,2,3,4,5]", col.GetValue()) - require.Equal(t, "text", col.GetMysqlType()) + require.Equal(t, "vector", col.GetMysqlType()) } } } diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go index 9a6fafd22ba..d8682314733 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go @@ -19,6 +19,7 @@ import ( "encoding/json" "testing" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/compression" @@ -88,7 +89,12 @@ func TestDMLE2E(t *testing.T) { colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID) decoded, ok := decodedColumns[colName] require.True(t, ok) - require.EqualValues(t, col.Value, decoded.Value) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value) + default: + require.EqualValues(t, v, decoded.Value) + } } err = encoder.AppendRowChangedEvent(ctx, "", updateEvent, func() {}) @@ -256,7 +262,12 @@ func TestCanalJSONClaimCheckE2E(t *testing.T) { colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID) decoded, ok := decodedColumns[colName] require.True(t, ok) - require.EqualValues(t, col.Value, decoded.Value) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value) + default: + require.EqualValues(t, v, decoded.Value) + } } } } @@ -667,7 +678,12 @@ func TestCanalJSONContentCompatibleE2E(t *testing.T) { colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID) decoded, ok := obtainedColumns[colName] require.True(t, ok) - require.EqualValues(t, col.Value, decoded.Value) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value) + default: + require.EqualValues(t, v, decoded.Value) + } } } @@ -722,7 +738,12 @@ func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID) decoded, ok := decodedColumns[colName] require.True(t, ok) - require.EqualValues(t, col.Value, decoded.Value) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value) + default: + require.EqualValues(t, v, decoded.Value) + } } _, hasNext, _ = decoder.HasNext() diff --git a/pkg/sink/codec/common/verify_checksum.go b/pkg/sink/codec/common/verify_checksum.go index 2557b39ef8c..49c0ba37724 100644 --- a/pkg/sink/codec/common/verify_checksum.go +++ b/pkg/sink/codec/common/verify_checksum.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/log" timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" @@ -215,6 +216,9 @@ func buildChecksumBytes(buf []byte, value interface{}, mysqlType byte) ([]byte, // this should not happen, does not take into the checksum calculation. case mysql.TypeNull, mysql.TypeGeometry: // do nothing + case mysql.TypeTiDBVectorFloat32: + vec, _ := types.ParseVectorFloat32(value.(string)) + buf = vec.SerializeTo(buf) default: return buf, errors.New("invalid type for the checksum calculation") } diff --git a/pkg/sink/codec/open/open_protocol_encoder_test.go b/pkg/sink/codec/open/open_protocol_encoder_test.go index f1833795999..5ad82506bd2 100644 --- a/pkg/sink/codec/open/open_protocol_encoder_test.go +++ b/pkg/sink/codec/open/open_protocol_encoder_test.go @@ -18,6 +18,7 @@ import ( "database/sql" "testing" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/compression" @@ -420,7 +421,12 @@ func TestE2EClaimCheckMessage(t *testing.T) { colName := insertEvent.TableInfo.ForceGetColumnName(column.ColumnID) decodedColumn, ok := decodedColumns[colName] require.True(t, ok) - require.Equal(t, column.Value, decodedColumn.Value, colName) + switch v := column.Value.(type) { + case types.VectorFloat32: + require.Equal(t, v.String(), decodedColumn.Value, colName) + default: + require.Equal(t, v, decodedColumn.Value, colName) + } } } diff --git a/pkg/sink/codec/simple/encoder_test.go b/pkg/sink/codec/simple/encoder_test.go index 5536bf4a8c4..59fb4fef7ea 100644 --- a/pkg/sink/codec/simple/encoder_test.go +++ b/pkg/sink/codec/simple/encoder_test.go @@ -27,6 +27,7 @@ import ( "github.com/golang/mock/gomock" timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/compression" @@ -1387,7 +1388,12 @@ func TestEncodeLargeEventsNormal(t *testing.T) { colName := event.TableInfo.ForceGetColumnName(col.ColumnID) decoded, ok := decodedColumns[colName] require.True(t, ok) - require.EqualValues(t, col.Value, decoded.Value) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value) + default: + require.EqualValues(t, v, decoded.Value) + } } decodedPreviousColumns := make(map[string]*model.ColumnData, len(decodedRow.PreColumns)) @@ -1399,7 +1405,12 @@ func TestEncodeLargeEventsNormal(t *testing.T) { colName := event.TableInfo.ForceGetColumnName(col.ColumnID) decoded, ok := decodedPreviousColumns[colName] require.True(t, ok) - require.EqualValues(t, col.Value, decoded.Value) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value) + default: + require.EqualValues(t, v, decoded.Value) + } } } } @@ -1541,7 +1552,12 @@ func TestLargerMessageHandleClaimCheck(t *testing.T) { colName := updateEvent.TableInfo.ForceGetColumnName(col.ColumnID) decoded, ok := decodedColumns[colName] require.True(t, ok) - require.EqualValues(t, col.Value, decoded.Value) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value, colName) + default: + require.EqualValues(t, v, decoded.Value, colName) + } } for _, column := range decodedRow.PreColumns { @@ -1552,7 +1568,12 @@ func TestLargerMessageHandleClaimCheck(t *testing.T) { colName := updateEvent.TableInfo.ForceGetColumnName(col.ColumnID) decoded, ok := decodedColumns[colName] require.True(t, ok) - require.EqualValues(t, col.Value, decoded.Value) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value, colName) + default: + require.EqualValues(t, v, decoded.Value, colName) + } } } } @@ -1723,11 +1744,18 @@ func TestLargeMessageHandleKeyOnly(t *testing.T) { case []byte: length := len(decoded.Value.([]uint8)) require.Equal(t, v[:length], decoded.Value, colName) + case types.VectorFloat32: + require.Equal(t, v.String(), decoded.Value, colName) default: - require.EqualValues(t, col.Value, decoded.Value, colName) + require.Equal(t, col.Value, decoded.Value, colName) } } else { - require.EqualValues(t, col.Value, decoded.Value, colName) + switch v := col.Value.(type) { + case []byte: + require.Equal(t, string(v), decoded.Value, colName) + default: + require.Equal(t, v, decoded.Value, colName) + } } } @@ -1746,11 +1774,20 @@ func TestLargeMessageHandleKeyOnly(t *testing.T) { case []byte: length := len(decoded.Value.([]uint8)) require.Equal(t, v[:length], decoded.Value, colName) + case types.VectorFloat32: + require.Equal(t, v.String(), decoded.Value, colName) default: - require.EqualValues(t, col.Value, decoded.Value, colName) + require.Equal(t, col.Value, decoded.Value, colName) } } else { - require.EqualValues(t, col.Value, decoded.Value, colName) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.Equal(t, v.String(), decoded.Value, colName) + case []byte: + require.Equal(t, string(v), decoded.Value, colName) + default: + require.Equal(t, v, decoded.Value, colName) + } } } } diff --git a/pkg/sink/codec/simple/message.go b/pkg/sink/codec/simple/message.go index 0453dcf8fa3..4b0ff09d107 100644 --- a/pkg/sink/codec/simple/message.go +++ b/pkg/sink/codec/simple/message.go @@ -402,7 +402,7 @@ func buildRowChangedEvent( err := common.VerifyChecksum(result, db) if err != nil || msg.Checksum.Corrupted { log.Warn("consumer detect checksum corrupted", - zap.String("schema", msg.Schema), zap.String("table", msg.Table)) + zap.String("schema", msg.Schema), zap.String("table", msg.Table), zap.Error(err)) return nil, cerror.ErrDecodeFailed.GenWithStackByArgs("checksum corrupted") } @@ -652,7 +652,6 @@ func encodeValue( if value == nil { return nil } - var err error switch ft.GetType() { case mysql.TypeBit: @@ -758,7 +757,18 @@ func decodeColumn(value interface{}, id int64, fieldType *types.FieldType) *mode case int64: value = uint64(v) } - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeInt24, mysql.TypeYear: + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeInt24: + switch v := value.(type) { + case string: + if mysql.HasUnsignedFlag(fieldType.GetFlag()) { + value, err = strconv.ParseUint(v, 10, 64) + } else { + value, err = strconv.ParseInt(v, 10, 64) + } + default: + value = v + } + case mysql.TypeYear: switch v := value.(type) { case string: value, err = strconv.ParseInt(v, 10, 64) @@ -768,9 +778,10 @@ func decodeColumn(value interface{}, id int64, fieldType *types.FieldType) *mode case mysql.TypeLonglong: switch v := value.(type) { case string: - value, err = strconv.ParseInt(v, 10, 64) - if err != nil { + if mysql.HasUnsignedFlag(fieldType.GetFlag()) { value, err = strconv.ParseUint(v, 10, 64) + } else { + value, err = strconv.ParseInt(v, 10, 64) } case map[string]interface{}: value = uint64(v["value"].(int64)) @@ -780,7 +791,9 @@ func decodeColumn(value interface{}, id int64, fieldType *types.FieldType) *mode case mysql.TypeFloat: switch v := value.(type) { case string: - value, err = strconv.ParseFloat(v, 32) + var val float64 + val, err = strconv.ParseFloat(v, 32) + value = float32(val) default: value = v } diff --git a/pkg/sink/codec/utils/test_utils.go b/pkg/sink/codec/utils/test_utils.go index 4385b411d18..18c38c89186 100644 --- a/pkg/sink/codec/utils/test_utils.go +++ b/pkg/sink/codec/utils/test_utils.go @@ -81,6 +81,7 @@ func NewLargeEvent4Test(t *testing.T, replicaConfig *config.ReplicaConfig) (*mod enumT enum('a', 'b', 'c') default 'b', setT set('a', 'b', 'c') default 'c', bitT bit(10) default b'1010101010', + vectorT vector(5), jsonT json)` ddlEvent := helper.DDL2Event(sql) @@ -138,6 +139,7 @@ func NewLargeEvent4Test(t *testing.T, replicaConfig *config.ReplicaConfig) (*mod 'a', 'b', 65, + '[1,2,3,4,5]', '{"key1": "value1"}')` insert := helper.DML2Event(sql, "test", "t") @@ -217,5 +219,6 @@ var LargeTableColumns = map[string]interface{}{ "enumT": []uint8("a"), "setT": []uint8("b"), "bitT": []uint8{65}, + "vectorT": []uint8("[1,2,3,4,5]"), "jsonT": []uint8("{\"key1\": \"value1\"}"), } diff --git a/tests/integration_tests/vector/run.sh b/tests/integration_tests/vector/run.sh index 3976b5be60e..aedcd2bccec 100755 --- a/tests/integration_tests/vector/run.sh +++ b/tests/integration_tests/vector/run.sh @@ -26,7 +26,7 @@ function run() { esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" case $SINK_TYPE in - kafka) run_kafka_consumer $WORK_DIR $SINK_URI;; + kafka) run_kafka_consumer $WORK_DIR $SINK_URI ;; storage) run_storage_consumer $WORK_DIR $SINK_URI "" "" ;; esac run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} From d91e22de8990d7f2ef3d73586a33b414cb7bb5f8 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Sat, 7 Sep 2024 23:19:49 +0800 Subject: [PATCH 22/37] chore --- cdc/sink/ddlsink/mysql/format_ddl.go | 16 +++++++++------- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 8 +++++--- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/format_ddl.go b/cdc/sink/ddlsink/mysql/format_ddl.go index 9fe07c3c89f..8332d18ff28 100644 --- a/cdc/sink/ddlsink/mysql/format_ddl.go +++ b/cdc/sink/ddlsink/mysql/format_ddl.go @@ -29,13 +29,15 @@ type visiter struct{} func (f *visiter) Enter(n ast.Node) (node ast.Node, skipChildren bool) { switch v := n.(type) { case *ast.ColumnDef: - switch v.Tp.GetType() { - case mysql.TypeTiDBVectorFloat32: - v.Tp.SetType(mysql.TypeLongBlob) - v.Tp.SetCharset("") - v.Tp.SetCollate("") - v.Tp.SetFlen(-1) - v.Options = []*ast.ColumnOption{} // clear COMMENT + if v.Tp != nil { + switch v.Tp.GetType() { + case mysql.TypeTiDBVectorFloat32: + v.Tp.SetType(mysql.TypeLongBlob) + v.Tp.SetCharset("") + v.Tp.SetCollate("") + v.Tp.SetFlen(-1) + v.Options = []*ast.ColumnOption{} // clear COMMENT + } } } return n, false diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index b29ee805a20..9670831c774 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -209,8 +209,10 @@ func (m *DDLSink) execDDL(pctx context.Context, ddl *model.DDLEvent) error { // Convert vector type to string type for unsupport database if m.needFormat { - ddl.Query = formatQuery(ddl.Query) - log.Warn("format ddl query", zap.String("query", ddl.Query), zap.String("collate", ddl.Collate), zap.String("charset", ddl.Charset)) + if newQuery := formatQuery(ddl.Query); newQuery != ddl.Query { + log.Warn("format ddl query", zap.String("newQuery", newQuery), zap.String("query", ddl.Query), zap.String("collate", ddl.Collate), zap.String("charset", ddl.Charset)) + ddl.Query = newQuery + } } failpoint.Inject("MySQLSinkExecDDLDelay", func() { @@ -299,7 +301,7 @@ func needFormatDDL(db *sql.DB, cfg *pmysql.Config) bool { serverInfo := version.ParseServerInfo(versionInfo) version := semver.New(defaultSupportVectorVersion) if !cfg.IsTiDB || serverInfo.ServerVersion.LessThan(*version) { - log.Error("downstream unsupport vector type. we convert it to longtext", zap.String("supportVersion", version.String()), zap.Bool("isTiDB", cfg.IsTiDB)) + log.Error("downstream unsupport vector type. we convert it to longtext", zap.String("version", serverInfo.ServerVersion.String()), zap.String("supportVersion", defaultSupportVectorVersion), zap.Bool("isTiDB", cfg.IsTiDB)) return true } return false From a483fc2439e8927e2dede82d5b2896311126c591 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Sun, 8 Sep 2024 19:55:04 +0800 Subject: [PATCH 23/37] fix --- .../pkg/meta/internal/sqlkv/sql_impl_test.go | 15 +++++++----- engine/pkg/orm/client_test.go | 24 +++++++++---------- engine/pkg/orm/model/logic_epoch_test.go | 4 ++-- 3 files changed, 23 insertions(+), 20 deletions(-) diff --git a/engine/pkg/meta/internal/sqlkv/sql_impl_test.go b/engine/pkg/meta/internal/sqlkv/sql_impl_test.go index caf244f200a..3b28196e550 100644 --- a/engine/pkg/meta/internal/sqlkv/sql_impl_test.go +++ b/engine/pkg/meta/internal/sqlkv/sql_impl_test.go @@ -58,6 +58,9 @@ func mockGetDBConn(t *testing.T, table string) (*sql.DB, sqlmock.Sqlmock) { mock.ExpectQuery("SELECT VERSION()"). WillReturnRows(sqlmock.NewRows([]string{"VERSION()"}). AddRow("5.7.35-log")) + mock.ExpectQuery(regexp.QuoteMeta("SELECT SCHEMA_NAME from Information_schema.SCHEMATA " + + "where SCHEMA_NAME LIKE ? ORDER BY SCHEMA_NAME=? DESC,SCHEMA_NAME limit 1")).WillReturnRows( + sqlmock.NewRows([]string{"SCHEMA_NAME"})) mock.ExpectExec(regexp.QuoteMeta(fmt.Sprintf("CREATE TABLE `%s` (`seq_id` bigint unsigned AUTO_INCREMENT,"+ "`created_at` datetime(3) NULL,`updated_at` datetime(3) NULL,`meta_key` varbinary(2048) not null,`meta_value` longblob,"+ "`job_id` varchar(64) not null,PRIMARY KEY (`seq_id`),UNIQUE INDEX `uidx_jk` (`job_id`,`meta_key`))", table))). @@ -145,8 +148,8 @@ func TestGet(t *testing.T) { }, mockExpectResFn: func(mock sqlmock.Sqlmock) { mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `fakeTable` WHERE job_id = ? AND "+ - "meta_key = ? ORDER BY `fakeTable`.`seq_id` LIMIT 1")). - WithArgs(fakeJob, []byte("key0")). + "meta_key = ? ORDER BY `fakeTable`.`seq_id` LIMIT ?")). + WithArgs(fakeJob, []byte("key0"), 1). WillReturnRows(sqlmock.NewRows([]string{"meta_key", "meta_value"})) }, }, @@ -167,8 +170,8 @@ func TestGet(t *testing.T) { }, mockExpectResFn: func(mock sqlmock.Sqlmock) { mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `fakeTable` WHERE job_id = ? AND "+ - "meta_key = ? ORDER BY `fakeTable`.`seq_id` LIMIT 1")). - WithArgs(fakeJob, []byte("key0")). + "meta_key = ? ORDER BY `fakeTable`.`seq_id` LIMIT ?")). + WithArgs(fakeJob, []byte("key0"), 1). WillReturnRows(sqlmock.NewRows([]string{"meta_key", "meta_value"}).AddRow("key0", "value0")) }, }, @@ -430,8 +433,8 @@ func TestSQLImplWithoutNamespace(t *testing.T) { cli.Put(ctx, "key0", "value0") mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `meta_kvs` WHERE job_id = ? AND "+ - "meta_key = ? ORDER BY `meta_kvs`.`seq_id` LIMIT 1")). - WithArgs("", []byte("key1")). + "meta_key = ? ORDER BY `meta_kvs`.`seq_id` LIMIT ?")). + WithArgs("", []byte("key1"), 1). WillReturnRows(sqlmock.NewRows([]string{"key", "value"})) cli.Get(ctx, "key1") diff --git a/engine/pkg/orm/client_test.go b/engine/pkg/orm/client_test.go index f89e857aab6..20d9b6d9949 100644 --- a/engine/pkg/orm/client_test.go +++ b/engine/pkg/orm/client_test.go @@ -206,7 +206,7 @@ func TestProject(t *testing.T) { Name: "tenant1", }, mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("111-222-333").WillReturnRows( + mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("111-222-333", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "id", "name", "seq_id", @@ -220,7 +220,7 @@ func TestProject(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("p111").WillReturnError( + mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("p111", 1).WillReturnError( errors.New("GetProjectByID error")) }, }, @@ -581,7 +581,7 @@ func TestJob(t *testing.T) { }, mockExpectResFn: func(mock sqlmock.Sqlmock) { expectedSQL := "SELECT * FROM `master_meta` WHERE id = ? AND `master_meta`.`deleted` IS NULL" - mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("j111").WillReturnRows( + mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("j111", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "id", "type", "state", "node_id", "address", "epoch", "config", "seq_id", "ext", @@ -597,7 +597,7 @@ func TestJob(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `master_meta` WHERE id").WithArgs("j111").WillReturnError( + mock.ExpectQuery("SELECT [*] FROM `master_meta` WHERE id").WithArgs("j111", 1).WillReturnError( errors.New("GetJobByID error")) }, }, @@ -836,7 +836,7 @@ func TestWorker(t *testing.T) { ExtBytes: []byte{0x11, 0x22}, }, mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", "w222").WillReturnRows( + mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", "w222", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "job_id", "id", "type", "state", "epoch", "error_message", "extend_bytes", "seq_id", @@ -852,7 +852,7 @@ func TestWorker(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", "w222").WillReturnError( + mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", "w222", 1).WillReturnError( errors.New("GetWorkerByID error")) }, }, @@ -1140,7 +1140,7 @@ func TestResource(t *testing.T) { Deleted: true, }, mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `resource_meta` WHERE job_id = ? AND id = ?")).WithArgs("j111", "r222").WillReturnRows( + mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `resource_meta` WHERE job_id = ? AND id = ?")).WithArgs("j111", "r222", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "id", "job_id", "worker_id", "executor_id", "deleted", "seq_id", @@ -1158,7 +1158,7 @@ func TestResource(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `resource_meta` WHERE job_id = ? AND id = ?")).WithArgs("j111", "r222").WillReturnError( + mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `resource_meta` WHERE job_id = ? AND id = ?")).WithArgs("j111", "r222", 1).WillReturnError( errors.New("GetResourceByID error")) }, }, @@ -1387,7 +1387,7 @@ func TestError(t *testing.T) { require.Nil(t, err) require.Len(t, res, 0) - mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("p111").WillReturnRows( + mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("p111", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "id", "name", "seq_id", @@ -1485,7 +1485,7 @@ func TestJobOp(t *testing.T) { }).AddRow(1)) mock.ExpectQuery( "SELECT [*] FROM `job_ops` WHERE job_id = ?"). - WithArgs("job-111").WillReturnRows( + WithArgs("job-111", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "op", "job_id", "seq_id", }).AddRow(createdAt, updatedAt, model.JobOpStatusCanceling, "job-111", 1)) @@ -1511,7 +1511,7 @@ func TestJobOp(t *testing.T) { }).AddRow(1)) mock.ExpectQuery( "SELECT [*] FROM `job_ops` WHERE job_id = ?"). - WithArgs("job-111").WillReturnRows( + WithArgs("job-111", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "op", "job_id", "seq_id", }).AddRow(createdAt, updatedAt, model.JobOpStatusCanceled, "job-111", 1)) @@ -1538,7 +1538,7 @@ func TestJobOp(t *testing.T) { }).AddRow(1)) mock.ExpectQuery( "SELECT [*] FROM `job_ops` WHERE job_id = ?"). - WithArgs("job-111").WillReturnRows( + WithArgs("job-111", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "op", "job_id", "seq_id", }).AddRow(createdAt, updatedAt, model.JobOpStatusNoop, "job-111", 1)) diff --git a/engine/pkg/orm/model/logic_epoch_test.go b/engine/pkg/orm/model/logic_epoch_test.go index ca55bceb6e3..b50643b13d3 100644 --- a/engine/pkg/orm/model/logic_epoch_test.go +++ b/engine/pkg/orm/model/logic_epoch_test.go @@ -91,8 +91,8 @@ func TestGenEpoch(t *testing.T) { mock.ExpectBegin() mock.ExpectExec(regexp.QuoteMeta("UPDATE `logic_epoches` SET `epoch`=epoch + ?,`updated_at`=? WHERE job_id = ?")). WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `logic_epoches` WHERE job_id = ? ORDER BY `logic_epoches`.`seq_id` LIMIT 1")). - WithArgs("fakeJob"). + mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `logic_epoches` WHERE job_id = ? ORDER BY `logic_epoches`.`seq_id` LIMIT ?")). + WithArgs("fakeJob", 1). WillReturnRows(sqlmock.NewRows([]string{"seq_id", "created_at", "updated_at", "job_id", "epoch"}). AddRow(1, createdAt, updatedAt, "fakeJob", 11)) mock.ExpectCommit() From d8ff62bc9c7d97c3ce21a87c1f0991431250a8a4 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Sun, 8 Sep 2024 20:31:24 +0800 Subject: [PATCH 24/37] . --- dm/syncer/expr_filter_group_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dm/syncer/expr_filter_group_test.go b/dm/syncer/expr_filter_group_test.go index 7bb15b68f01..0cc161981f4 100644 --- a/dm/syncer/expr_filter_group_test.go +++ b/dm/syncer/expr_filter_group_test.go @@ -17,6 +17,7 @@ import ( "context" "testing" + "github.com/pingcap/errors" ddl2 "github.com/pingcap/tidb/pkg/ddl" context2 "github.com/pingcap/tidb/pkg/expression/context" "github.com/pingcap/tidb/pkg/parser/ast" @@ -440,7 +441,7 @@ create table t ( require.NoError(t, err) require.Len(t, exprs, 1) expr := exprs[0] - require.Equal(t, "0", expr.StringWithCtx(context2.EmptyParamValues)) + require.Equal(t, "0", expr.StringWithCtx(context2.EmptyParamValues, errors.RedactLogDisable)) // skip nothing skip, err := SkipDMLByExpression(sessCtx, []interface{}{0}, expr, ti.Columns) From d195be6c76dc0a2e127d0e95fedcb3089ab478b6 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Sun, 8 Sep 2024 21:36:51 +0800 Subject: [PATCH 25/37] lint --- cdc/sink/ddlsink/mysql/format_ddl_test.go | 4 ++-- cdc/sink/util/vector.go | 6 ++++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/format_ddl_test.go b/cdc/sink/ddlsink/mysql/format_ddl_test.go index beb5bc7e77b..759af5e7656 100644 --- a/cdc/sink/ddlsink/mysql/format_ddl_test.go +++ b/cdc/sink/ddlsink/mysql/format_ddl_test.go @@ -26,7 +26,7 @@ import ( func TestFormatQuery(t *testing.T) { sql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` VECTOR(5));" - expectSql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT)" + expectSQL := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT)" p := parser.New() stmt, err := p.ParseOneStmt(sql, "", "") if err != nil { @@ -39,5 +39,5 @@ func TestFormatQuery(t *testing.T) { if err = stmt.Restore(restoreCtx); err != nil { log.Error("format query restore failed", zap.Error(err)) } - require.Equal(t, buf.String(), expectSql) + require.Equal(t, buf.String(), expectSQL) } diff --git a/cdc/sink/util/vector.go b/cdc/sink/util/vector.go index 1138a64517e..1f6fa6922b1 100644 --- a/cdc/sink/util/vector.go +++ b/cdc/sink/util/vector.go @@ -15,11 +15,11 @@ package util import ( "encoding/json" - "log" "reflect" "strconv" "unsafe" + "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/types" "go.uber.org/zap" ) @@ -36,7 +36,7 @@ import ( // return errors.Errorf("currently HNSW index can not be removed") // } // if oldCol.FieldType.GetFlen() != newCol.FieldType.GetFlen() { -// return errors.New("cannot modify vector column's dimention when HNSW index is defined") +// return errors.New("cannot modify vector column's dimension when HNSW index is defined") // } // if oldCol.FieldType.GetType() != newCol.FieldType.GetType() { // return errors.New("cannot modify column data type when HNSW index is defined") @@ -47,6 +47,7 @@ import ( // return nil // } +// ParseVectorFromElement parse element to VectorFloat32 func ParseVectorFromElement(values []float32) (types.VectorFloat32, error) { dim := len(values) if err := types.CheckVectorDimValid(dim); err != nil { @@ -57,6 +58,7 @@ func ParseVectorFromElement(values []float32) (types.VectorFloat32, error) { return vec, nil } +// VectorElement2String parse elements to string func VectorElement2String(elements []interface{}) string { buf := make([]byte, 0, 2+len(elements)*2) buf = append(buf, '[') From 0054111e2a5ae7f7a02343d0a2044f1d8ef02146 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Mon, 9 Sep 2024 12:34:57 +0800 Subject: [PATCH 26/37] debug --- go.mod | 6 ++++-- go.sum | 17 +++++++++-------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/go.mod b/go.mod index 109638e1a3b..0643b691dce 100644 --- a/go.mod +++ b/go.mod @@ -398,7 +398,7 @@ require ( ) // Fix https://github.com/pingcap/tiflow/issues/4961 -replace github.com/benbjohnson/clock v1.3.0 => github.com/benbjohnson/clock v1.1.0 +replace github.com/benbjohnson/clock v1.3.5 => github.com/benbjohnson/clock v1.1.0 // copy from TiDB replace go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac @@ -410,7 +410,9 @@ replace github.com/tildeleb/hashland => leb.io/hashland v0.1.5 replace github.com/chaos-mesh/go-sqlsmith => github.com/PingCAP-QE/go-sqlsmith v0.0.0-20231213065948-336e064b488d -replace gorm.io/driver/mysql v1.4.5 => gorm.io/driver/mysql v1.3.3 +replace gorm.io/driver/mysql v1.5.7 => gorm.io/driver/mysql v1.3.3 + +replace gorm.io/gorm v1.25.11 => gorm.io/gorm v1.24.5 // TODO: `sourcegraph.com/sourcegraph/appdash` has been archived, and the original host has been removed. // Please remove these dependencies. diff --git a/go.sum b/go.sum index bd0d0709037..c2926405a99 100644 --- a/go.sum +++ b/go.sum @@ -152,9 +152,9 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.20.1 h1:U7h9CPoyMfVoN5jUglB0LglCMP10 github.com/aws/aws-sdk-go-v2/service/sts v1.20.1/go.mod h1:BUHusg4cOA1TFGegj7x8/eoWrbdHzJfoMrXcbMQAG0k= github.com/aws/smithy-go v1.13.5 h1:hgz0X/DX0dGqTYpGALqXJoRKRj5oQ7150i5FdTePzO8= github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/benbjohnson/clock v1.3.5 h1:VvXlSJBzZpA/zum6Sj74hxwYI2DIxRWuNIoXAzHZz5o= -github.com/benbjohnson/clock v1.3.5/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= +github.com/benbjohnson/clock v1.3.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.4.0 h1:+YZ8ePm+He2pU3dZlIZiOeAKfrBkXi1lSrXJ/Xzgbu8= @@ -425,7 +425,7 @@ github.com/go-resty/resty/v2 v2.11.0 h1:i7jMfNOJYMp69lq7qozJP+bjgzfAzeOhuGlyDrqx github.com/go-resty/resty/v2 v2.11.0/go.mod h1:iiP/OpA0CkcL3IGt1O0+/SIItFUbkkyw5BGXiVdTu+A= github.com/go-session/session v3.1.2+incompatible/go.mod h1:8B3iivBQjrz/JtC68Np2T1yBBLxTan3mn/3OM0CyRt0= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= -github.com/go-sql-driver/mysql v1.7.0/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= +github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= 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= @@ -637,6 +637,7 @@ github.com/jfcg/sorty/v2 v2.1.0/go.mod h1:JpcSKlmtGOOAGyTdWN2ErjvxeMSJVYBsylAKep github.com/jhump/protoreflect v1.6.0/go.mod h1:eaTn3RZAmMBcV0fifFvlm6VHNz3wSkYyXYWUh7ymB74= github.com/jinzhu/inflection v1.0.0 h1:K317FqzuhWc8YvSVlFMCCUb36O/S9MCKRDI7QkRKD/E= github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= +github.com/jinzhu/now v1.1.4/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= @@ -1728,11 +1729,11 @@ gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C gopkg.in/yaml.v3 v3.0.0-20220512140231-539c8e751b99/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= -gorm.io/driver/mysql v1.5.7 h1:MndhOPYOfEp2rHKgkZIhJ16eVUIRf2HmzgoPmh7FCWo= -gorm.io/driver/mysql v1.5.7/go.mod h1:sEtPWMiqiN1N1cMXoXmBbd8C6/l+TESwriotuRRpkDM= -gorm.io/gorm v1.25.7/go.mod h1:hbnx/Oo0ChWMn1BIhpy1oYozzpM15i4YPuHDmfYtwg8= -gorm.io/gorm v1.25.11 h1:/Wfyg1B/je1hnDx3sMkX+gAlxrlZpn6X0BXRlwXlvHg= -gorm.io/gorm v1.25.11/go.mod h1:xh7N7RHfYlNc5EmcI/El95gXusucDrQnHXe0+CgWcLQ= +gorm.io/driver/mysql v1.3.3 h1:jXG9ANrwBc4+bMvBcSl8zCfPBaVoPyBEBshA8dA93X8= +gorm.io/driver/mysql v1.3.3/go.mod h1:ChK6AHbHgDCFZyJp0F+BmVGb06PSIoh9uVYKAlRbb2U= +gorm.io/gorm v1.23.1/go.mod h1:l2lP/RyAtc1ynaTjFksBde/O8v9oOGIApu2/xRitmZk= +gorm.io/gorm v1.24.5 h1:g6OPREKqqlWq4kh/3MCQbZKImeB9e6Xgc4zD+JgNZGE= +gorm.io/gorm v1.24.5/go.mod h1:DVrVomtaYTbqs7gB/x2uVvqnXzv0nqjB396B8cG4dBA= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= From b43a678aa3d7b580b6c1d9bc0353b1c1a4497b9c Mon Sep 17 00:00:00 2001 From: wk989898 Date: Mon, 9 Sep 2024 13:36:14 +0800 Subject: [PATCH 27/37] revert --- cdc/entry/mounter_test.go | 1 + cdc/entry/schema_storage_test.go | 5 ++- cdc/entry/schema_test_helper.go | 1 + .../pkg/meta/internal/sqlkv/sql_impl_test.go | 15 ++++----- engine/pkg/orm/client_test.go | 32 +++++++++---------- engine/pkg/orm/model/logic_epoch_test.go | 4 +-- pkg/election/storage_orm_test.go | 24 +++++++------- 7 files changed, 42 insertions(+), 40 deletions(-) diff --git a/cdc/entry/mounter_test.go b/cdc/entry/mounter_test.go index 680b656fbe1..760481ebd80 100644 --- a/cdc/entry/mounter_test.go +++ b/cdc/entry/mounter_test.go @@ -269,6 +269,7 @@ func testMounterDisableOldValue(t *testing.T, tc struct { ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { // we can update the tidb config here }) + session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() diff --git a/cdc/entry/schema_storage_test.go b/cdc/entry/schema_storage_test.go index 6128a557810..1e8e96e62dc 100644 --- a/cdc/entry/schema_storage_test.go +++ b/cdc/entry/schema_storage_test.go @@ -672,7 +672,7 @@ func TestCreateSnapFromMeta(t *testing.T) { store, err := mockstore.NewMockStore() require.Nil(t, err) defer store.Close() //nolint:errcheck - + session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() @@ -706,6 +706,7 @@ func TestExplicitTables(t *testing.T) { require.Nil(t, err) defer store.Close() //nolint:errcheck + session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() @@ -855,6 +856,7 @@ func TestSchemaStorage(t *testing.T) { ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { conf.AlterPrimaryKey = true }) + session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() @@ -946,6 +948,7 @@ func TestHandleKey(t *testing.T) { require.Nil(t, err) defer store.Close() //nolint:errcheck + session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) defer domain.Close() diff --git a/cdc/entry/schema_test_helper.go b/cdc/entry/schema_test_helper.go index 324fc2bfbd7..9d9e14faf45 100644 --- a/cdc/entry/schema_test_helper.go +++ b/cdc/entry/schema_test_helper.go @@ -61,6 +61,7 @@ func NewSchemaTestHelperWithReplicaConfig( ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { conf.AlterPrimaryKey = true }) + session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.NoError(t, err) domain.SetStatsUpdating(true) diff --git a/engine/pkg/meta/internal/sqlkv/sql_impl_test.go b/engine/pkg/meta/internal/sqlkv/sql_impl_test.go index 3b28196e550..caf244f200a 100644 --- a/engine/pkg/meta/internal/sqlkv/sql_impl_test.go +++ b/engine/pkg/meta/internal/sqlkv/sql_impl_test.go @@ -58,9 +58,6 @@ func mockGetDBConn(t *testing.T, table string) (*sql.DB, sqlmock.Sqlmock) { mock.ExpectQuery("SELECT VERSION()"). WillReturnRows(sqlmock.NewRows([]string{"VERSION()"}). AddRow("5.7.35-log")) - mock.ExpectQuery(regexp.QuoteMeta("SELECT SCHEMA_NAME from Information_schema.SCHEMATA " + - "where SCHEMA_NAME LIKE ? ORDER BY SCHEMA_NAME=? DESC,SCHEMA_NAME limit 1")).WillReturnRows( - sqlmock.NewRows([]string{"SCHEMA_NAME"})) mock.ExpectExec(regexp.QuoteMeta(fmt.Sprintf("CREATE TABLE `%s` (`seq_id` bigint unsigned AUTO_INCREMENT,"+ "`created_at` datetime(3) NULL,`updated_at` datetime(3) NULL,`meta_key` varbinary(2048) not null,`meta_value` longblob,"+ "`job_id` varchar(64) not null,PRIMARY KEY (`seq_id`),UNIQUE INDEX `uidx_jk` (`job_id`,`meta_key`))", table))). @@ -148,8 +145,8 @@ func TestGet(t *testing.T) { }, mockExpectResFn: func(mock sqlmock.Sqlmock) { mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `fakeTable` WHERE job_id = ? AND "+ - "meta_key = ? ORDER BY `fakeTable`.`seq_id` LIMIT ?")). - WithArgs(fakeJob, []byte("key0"), 1). + "meta_key = ? ORDER BY `fakeTable`.`seq_id` LIMIT 1")). + WithArgs(fakeJob, []byte("key0")). WillReturnRows(sqlmock.NewRows([]string{"meta_key", "meta_value"})) }, }, @@ -170,8 +167,8 @@ func TestGet(t *testing.T) { }, mockExpectResFn: func(mock sqlmock.Sqlmock) { mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `fakeTable` WHERE job_id = ? AND "+ - "meta_key = ? ORDER BY `fakeTable`.`seq_id` LIMIT ?")). - WithArgs(fakeJob, []byte("key0"), 1). + "meta_key = ? ORDER BY `fakeTable`.`seq_id` LIMIT 1")). + WithArgs(fakeJob, []byte("key0")). WillReturnRows(sqlmock.NewRows([]string{"meta_key", "meta_value"}).AddRow("key0", "value0")) }, }, @@ -433,8 +430,8 @@ func TestSQLImplWithoutNamespace(t *testing.T) { cli.Put(ctx, "key0", "value0") mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `meta_kvs` WHERE job_id = ? AND "+ - "meta_key = ? ORDER BY `meta_kvs`.`seq_id` LIMIT ?")). - WithArgs("", []byte("key1"), 1). + "meta_key = ? ORDER BY `meta_kvs`.`seq_id` LIMIT 1")). + WithArgs("", []byte("key1")). WillReturnRows(sqlmock.NewRows([]string{"key", "value"})) cli.Get(ctx, "key1") diff --git a/engine/pkg/orm/client_test.go b/engine/pkg/orm/client_test.go index 20d9b6d9949..c1e77ded266 100644 --- a/engine/pkg/orm/client_test.go +++ b/engine/pkg/orm/client_test.go @@ -206,7 +206,7 @@ func TestProject(t *testing.T) { Name: "tenant1", }, mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("111-222-333", 1).WillReturnRows( + mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("111-222-333").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "id", "name", "seq_id", @@ -220,7 +220,7 @@ func TestProject(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("p111", 1).WillReturnError( + mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("p111").WillReturnError( errors.New("GetProjectByID error")) }, }, @@ -581,7 +581,7 @@ func TestJob(t *testing.T) { }, mockExpectResFn: func(mock sqlmock.Sqlmock) { expectedSQL := "SELECT * FROM `master_meta` WHERE id = ? AND `master_meta`.`deleted` IS NULL" - mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("j111", 1).WillReturnRows( + mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("j111").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "id", "type", "state", "node_id", "address", "epoch", "config", "seq_id", "ext", @@ -597,7 +597,7 @@ func TestJob(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `master_meta` WHERE id").WithArgs("j111", 1).WillReturnError( + mock.ExpectQuery("SELECT [*] FROM `master_meta` WHERE id").WithArgs("j111").WillReturnError( errors.New("GetJobByID error")) }, }, @@ -672,7 +672,7 @@ func TestJob(t *testing.T) { }, mockExpectResFn: func(mock sqlmock.Sqlmock) { expectedSQL := "SELECT * FROM `master_meta` WHERE (project_id = ? AND state = ?) AND `master_meta`.`deleted` IS NULL" - mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("p111", 1).WillReturnRows( + mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("p111").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "id", "type", "state", "node_id", "address", "epoch", "config", "seq_id", "ext", @@ -689,7 +689,7 @@ func TestJob(t *testing.T) { err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { expectedSQL := "SELECT * FROM `master_meta` WHERE (project_id = ? AND state = ?) AND `master_meta`.`deleted` IS NULL" - mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("p111", 1).WillReturnError( + mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("p111").WillReturnError( errors.New("QueryJobsByState error")) }, }, @@ -836,7 +836,7 @@ func TestWorker(t *testing.T) { ExtBytes: []byte{0x11, 0x22}, }, mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", "w222", 1).WillReturnRows( + mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", "w222").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "job_id", "id", "type", "state", "epoch", "error_message", "extend_bytes", "seq_id", @@ -852,7 +852,7 @@ func TestWorker(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", "w222", 1).WillReturnError( + mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", "w222").WillReturnError( errors.New("GetWorkerByID error")) }, }, @@ -924,7 +924,7 @@ func TestWorker(t *testing.T) { }, }, mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", 1).WillReturnRows( + mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "job_id", "id", "type", "state", "epoch", "error_message", "extend_bytes", "seq_id", @@ -940,7 +940,7 @@ func TestWorker(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", 1).WillReturnError( + mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111").WillReturnError( errors.New("QueryWorkersByState error")) }, }, @@ -1140,7 +1140,7 @@ func TestResource(t *testing.T) { Deleted: true, }, mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `resource_meta` WHERE job_id = ? AND id = ?")).WithArgs("j111", "r222", 1).WillReturnRows( + mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `resource_meta` WHERE job_id = ? AND id = ?")).WithArgs("j111", "r222").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "id", "job_id", "worker_id", "executor_id", "deleted", "seq_id", @@ -1158,7 +1158,7 @@ func TestResource(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `resource_meta` WHERE job_id = ? AND id = ?")).WithArgs("j111", "r222", 1).WillReturnError( + mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `resource_meta` WHERE job_id = ? AND id = ?")).WithArgs("j111", "r222").WillReturnError( errors.New("GetResourceByID error")) }, }, @@ -1387,7 +1387,7 @@ func TestError(t *testing.T) { require.Nil(t, err) require.Len(t, res, 0) - mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("p111", 1).WillReturnRows( + mock.ExpectQuery("SELECT [*] FROM `project_infos` WHERE id").WithArgs("p111").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "id", "name", "seq_id", @@ -1485,7 +1485,7 @@ func TestJobOp(t *testing.T) { }).AddRow(1)) mock.ExpectQuery( "SELECT [*] FROM `job_ops` WHERE job_id = ?"). - WithArgs("job-111", 1).WillReturnRows( + WithArgs("job-111").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "op", "job_id", "seq_id", }).AddRow(createdAt, updatedAt, model.JobOpStatusCanceling, "job-111", 1)) @@ -1511,7 +1511,7 @@ func TestJobOp(t *testing.T) { }).AddRow(1)) mock.ExpectQuery( "SELECT [*] FROM `job_ops` WHERE job_id = ?"). - WithArgs("job-111", 1).WillReturnRows( + WithArgs("job-111").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "op", "job_id", "seq_id", }).AddRow(createdAt, updatedAt, model.JobOpStatusCanceled, "job-111", 1)) @@ -1538,7 +1538,7 @@ func TestJobOp(t *testing.T) { }).AddRow(1)) mock.ExpectQuery( "SELECT [*] FROM `job_ops` WHERE job_id = ?"). - WithArgs("job-111", 1).WillReturnRows( + WithArgs("job-111").WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "op", "job_id", "seq_id", }).AddRow(createdAt, updatedAt, model.JobOpStatusNoop, "job-111", 1)) diff --git a/engine/pkg/orm/model/logic_epoch_test.go b/engine/pkg/orm/model/logic_epoch_test.go index b50643b13d3..ca55bceb6e3 100644 --- a/engine/pkg/orm/model/logic_epoch_test.go +++ b/engine/pkg/orm/model/logic_epoch_test.go @@ -91,8 +91,8 @@ func TestGenEpoch(t *testing.T) { mock.ExpectBegin() mock.ExpectExec(regexp.QuoteMeta("UPDATE `logic_epoches` SET `epoch`=epoch + ?,`updated_at`=? WHERE job_id = ?")). WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `logic_epoches` WHERE job_id = ? ORDER BY `logic_epoches`.`seq_id` LIMIT ?")). - WithArgs("fakeJob", 1). + mock.ExpectQuery(regexp.QuoteMeta("SELECT * FROM `logic_epoches` WHERE job_id = ? ORDER BY `logic_epoches`.`seq_id` LIMIT 1")). + WithArgs("fakeJob"). WillReturnRows(sqlmock.NewRows([]string{"seq_id", "created_at", "updated_at", "job_id", "epoch"}). AddRow(1, createdAt, updatedAt, "fakeJob", 11)) mock.ExpectCommit() diff --git a/pkg/election/storage_orm_test.go b/pkg/election/storage_orm_test.go index ae75a1fbe27..443610f5e92 100644 --- a/pkg/election/storage_orm_test.go +++ b/pkg/election/storage_orm_test.go @@ -34,9 +34,9 @@ func newORMStorageAndMock(t *testing.T) (*ORMStorage, sqlmock.Sqlmock) { db, err := ormUtil.NewGormDB(backendDB, "mysql") require.NoError(t, err) mock.ExpectQuery("SELECT SCHEMA_NAME from Information_schema.SCHEMATA " + - "where SCHEMA_NAME LIKE ? ORDER BY SCHEMA_NAME=? DESC,SCHEMA_NAME limit 1").WillReturnRows( + "where SCHEMA_NAME LIKE ? ORDER BY SCHEMA_NAME=? DESC limit 1").WillReturnRows( sqlmock.NewRows([]string{"SCHEMA_NAME"})) - mock.ExpectExec("CREATE TABLE `test` (`id` int(10) unsigned AUTO_INCREMENT,`leader_id` text NOT NULL," + + mock.ExpectExec("CREATE TABLE `test` (`id` int(10) unsigned,`leader_id` text NOT NULL," + "`record` text,`version` bigint(20) unsigned NOT NULL,PRIMARY KEY (`id`))"). WillReturnResult(sqlmock.NewResult(0, 0)) @@ -49,8 +49,8 @@ func newORMStorageAndMock(t *testing.T) (*ORMStorage, sqlmock.Sqlmock) { func TestORMStorageGetEmptyRecord(t *testing.T) { s, mock := newORMStorageAndMock(t) - mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT ?"). - WithArgs(1, 1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"})) + mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT 1"). + WithArgs(1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"})) record, err := s.Get(context.Background()) require.NoError(t, err) require.Equal(t, &Record{}, record) @@ -76,8 +76,8 @@ func TestORMStorageGetExistingRecord(t *testing.T) { recordBytes, err := json.Marshal(expectedRecord) require.NoError(t, err) - mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT ?"). - WithArgs(1, 1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"}). + mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT 1"). + WithArgs(1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"}). AddRow(1, "id1", recordBytes, 1)) record, err := s.Get(context.Background()) require.NoError(t, err) @@ -189,8 +189,8 @@ func TestORMStorageTxnWithLeaderCheck(t *testing.T) { s, mock := newORMStorageAndMock(t) mock.ExpectBegin() - mock.ExpectQuery("SELECT `leader_id` FROM `test` WHERE id = ? and leader_id = ? LIMIT ? LOCK IN SHARE MODE"). - WithArgs(leaderRowID, "leader1", 1).WillReturnRows(sqlmock.NewRows([]string{"leader_id"})) + mock.ExpectQuery("SELECT `leader_id` FROM `test` WHERE id = ? and leader_id = ? LIMIT 1 LOCK IN SHARE MODE"). + WithArgs(leaderRowID, "leader1").WillReturnRows(sqlmock.NewRows([]string{"leader_id"})) mock.ExpectRollback() doNothing := func(*gorm.DB) error { return nil @@ -199,11 +199,11 @@ func TestORMStorageTxnWithLeaderCheck(t *testing.T) { require.ErrorIs(t, err, errors.ErrElectorNotLeader) mock.ExpectBegin() - mock.ExpectQuery("SELECT `leader_id` FROM `test` WHERE id = ? and leader_id = ? LIMIT ? LOCK IN SHARE MODE"). - WithArgs(leaderRowID, "leader1", 1). + mock.ExpectQuery("SELECT `leader_id` FROM `test` WHERE id = ? and leader_id = ? LIMIT 1 LOCK IN SHARE MODE"). + WithArgs(leaderRowID, "leader1"). WillReturnRows(sqlmock.NewRows([]string{"leader_id"}).AddRow("leader1")) - mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT ?"). - WithArgs(1, 1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"})) + mock.ExpectQuery("SELECT * FROM `test` WHERE id = ? LIMIT 1"). + WithArgs(1).WillReturnRows(sqlmock.NewRows([]string{"id", "leader_id", "record", "version"})) mock.ExpectCommit() doTxn := func(tx *gorm.DB) error { _, err := s.Get(context.Background()) From 0f38fa890019fff5dbea5ca19ad31c83a403b545 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Mon, 9 Sep 2024 13:41:19 +0800 Subject: [PATCH 28/37] revert --- engine/pkg/orm/client_test.go | 8 ++++---- pkg/filter/filter_test_helper.go | 1 + 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/engine/pkg/orm/client_test.go b/engine/pkg/orm/client_test.go index c1e77ded266..f89e857aab6 100644 --- a/engine/pkg/orm/client_test.go +++ b/engine/pkg/orm/client_test.go @@ -672,7 +672,7 @@ func TestJob(t *testing.T) { }, mockExpectResFn: func(mock sqlmock.Sqlmock) { expectedSQL := "SELECT * FROM `master_meta` WHERE (project_id = ? AND state = ?) AND `master_meta`.`deleted` IS NULL" - mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("p111").WillReturnRows( + mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("p111", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "id", "type", "state", "node_id", "address", "epoch", "config", "seq_id", "ext", @@ -689,7 +689,7 @@ func TestJob(t *testing.T) { err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { expectedSQL := "SELECT * FROM `master_meta` WHERE (project_id = ? AND state = ?) AND `master_meta`.`deleted` IS NULL" - mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("p111").WillReturnError( + mock.ExpectQuery(regexp.QuoteMeta(expectedSQL)).WithArgs("p111", 1).WillReturnError( errors.New("QueryJobsByState error")) }, }, @@ -924,7 +924,7 @@ func TestWorker(t *testing.T) { }, }, mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111").WillReturnRows( + mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", 1).WillReturnRows( sqlmock.NewRows([]string{ "created_at", "updated_at", "project_id", "job_id", "id", "type", "state", "epoch", "error_message", "extend_bytes", "seq_id", @@ -940,7 +940,7 @@ func TestWorker(t *testing.T) { }, err: errors.ErrMetaOpFail.GenWithStackByArgs(), mockExpectResFn: func(mock sqlmock.Sqlmock) { - mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111").WillReturnError( + mock.ExpectQuery("SELECT [*] FROM `worker_statuses` WHERE job_id").WithArgs("j111", 1).WillReturnError( errors.New("QueryWorkersByState error")) }, }, diff --git a/pkg/filter/filter_test_helper.go b/pkg/filter/filter_test_helper.go index a30915f5c38..30f39233be1 100644 --- a/pkg/filter/filter_test_helper.go +++ b/pkg/filter/filter_test_helper.go @@ -46,6 +46,7 @@ func newTestHelper(t *testing.T) *testHelper { ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { conf.AlterPrimaryKey = true }) + session.DisableStats4Test() domain, err := session.BootstrapSession(store) require.Nil(t, err) domain.SetStatsUpdating(true) From cae23098a99fbe9c08a12c5749d541e42a0782af Mon Sep 17 00:00:00 2001 From: wk989898 Date: Wed, 11 Sep 2024 16:05:38 +0800 Subject: [PATCH 29/37] update sync_diff_inspector_url --- scripts/download-integration-test-binaries.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/download-integration-test-binaries.sh b/scripts/download-integration-test-binaries.sh index 60731df7ee6..fceeb049842 100755 --- a/scripts/download-integration-test-binaries.sh +++ b/scripts/download-integration-test-binaries.sh @@ -142,7 +142,7 @@ function download_binaries() { minio_download_url="${file_server_url}/download/minio.tar.gz" go_ycsb_download_url="${file_server_url}/download/builds/pingcap/go-ycsb/test-br/go-ycsb" etcd_download_url="${file_server_url}/download/builds/pingcap/cdc/etcd-v3.4.7-linux-amd64.tar.gz" - sync_diff_inspector_url="${file_server_url}/download/builds/pingcap/cdc/sync_diff_inspector_hash-79f1fd1e_linux-amd64.tar.gz" + sync_diff_inspector_url="${file_server_url}/download/builds/pingcap/cdc/sync_diff_inspector_hash-a129f096_linux-amd64.tar.gz" jq_download_url="${file_server_url}/download/builds/pingcap/test/jq-1.6/jq-linux64" schema_registry_url="${file_server_url}/download/builds/pingcap/cdc/schema-registry.tar.gz" From f072df32a7e1b0b559a9ecdd3295ea5a458be43c Mon Sep 17 00:00:00 2001 From: wk989898 Date: Wed, 11 Sep 2024 16:57:57 +0800 Subject: [PATCH 30/37] update test --- cdc/sink/dmlsink/txn/mysql/dml_test.go | 19 ++++++++ cdc/sink/dmlsink/txn/mysql/mysql_test.go | 43 +------------------ scripts/download-integration-test-binaries.sh | 2 +- 3 files changed, 22 insertions(+), 42 deletions(-) diff --git a/cdc/sink/dmlsink/txn/mysql/dml_test.go b/cdc/sink/dmlsink/txn/mysql/dml_test.go index dab7c4b104f..fd2807a7510 100644 --- a/cdc/sink/dmlsink/txn/mysql/dml_test.go +++ b/cdc/sink/dmlsink/txn/mysql/dml_test.go @@ -18,7 +18,9 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util" "github.com/stretchr/testify/require" ) @@ -709,6 +711,23 @@ func TestMapReplace(t *testing.T) { []byte("你好,世界"), }, }, + { + quoteTable: "`test`.`t1`", + cols: []*model.Column{ + { + Name: "a", + Type: mysql.TypeTiDBVectorFloat32, + Value: util.Must(types.ParseVectorFloat32("[1.0,-2,0.3,-4.4,55]")), + }, + { + Name: "b", + Type: mysql.TypeTiDBVectorFloat32, + Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), + }, + }, + expectedQuery: "REPLACE INTO `test`.`t1` (`a`,`b`) VALUES ", + expectedArgs: []interface{}{"[1,-2,0.3,-4.4,55]", "[1,2,3,4,5]"}, + }, } for _, tc := range testCases { // multiple times to verify the stability of column sequence in query string diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index 7edcc7baa19..ef0e6120f42 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -1492,56 +1492,17 @@ func TestPrepareBatchDMLs(t *testing.T) { }}, tableInfoWithVector), ApproximateDataSize: 10, }, - { - StartTs: 418658114257813516, - CommitTs: 418658114257813517, - TableInfo: tableInfoWithVector, - PreColumns: model.Columns2ColumnDatas([]*model.Column{{ - Name: "a1", - Value: 1, - }, { - Name: "a3", - Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), - }}, tableInfoWithVector), - Columns: model.Columns2ColumnDatas([]*model.Column{{ - Name: "a1", - Value: 3, - }, { - Name: "a3", - Value: util.Must(types.ParseVectorFloat32("[1.1,-2,3.33,-4.12,-5]")), - }}, tableInfoWithVector), - ApproximateDataSize: 10, - }, - { - StartTs: 418658114257813516, - CommitTs: 418658114257813517, - TableInfo: tableInfoWithVector, - PreColumns: model.Columns2ColumnDatas([]*model.Column{{ - Name: "a1", - Value: 3, - }, { - Name: "a3", - Value: util.Must(types.ParseVectorFloat32("[1.1,-2,3.33,-4.12,-5]")), - }}, tableInfoWithVector), - ApproximateDataSize: 10, - }, }, expected: &preparedDMLs{ startTs: []model.Ts{418658114257813516}, sqls: []string{ - "DELETE FROM `common_1`.`uk_without_pk` WHERE (`a1` = ? AND `a3` = ?)", - "UPDATE `common_1`.`uk_without_pk` SET `a1`=CASE WHEN " + - "`a1` = ? AND `a3` = ? THEN ? END, `a3`=CASE WHEN " + - "`a1` = ? AND `a3` = ? THEN ? END WHERE (`a1` = ? AND `a3` = ?)", "INSERT INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)", }, values: [][]interface{}{ - {3, "[1.1,-2,3.33,-4.12,-5]"}, - {1, "[1,2,3,4,5]", 3, 1, "[1,2,3,4,5]", "[1.1,-2,3.33,-4.12,-5]", 1, "[1,2,3,4,5]"}, {1, "[1,2,3,4,5]"}, }, - rowCount: 3, - approximateSize: 325, + rowCount: 1, + approximateSize: 73, }, }, } diff --git a/scripts/download-integration-test-binaries.sh b/scripts/download-integration-test-binaries.sh index fceeb049842..80f20e27c68 100755 --- a/scripts/download-integration-test-binaries.sh +++ b/scripts/download-integration-test-binaries.sh @@ -141,7 +141,7 @@ function download_binaries() { tiflash_download_url="${file_server_url}/download/builds/pingcap/tiflash/${tiflash_branch}/${tiflash_sha1}/centos7/tiflash.tar.gz" minio_download_url="${file_server_url}/download/minio.tar.gz" go_ycsb_download_url="${file_server_url}/download/builds/pingcap/go-ycsb/test-br/go-ycsb" - etcd_download_url="${file_server_url}/download/builds/pingcap/cdc/etcd-v3.4.7-linux-amd64.tar.gz" + etcd_download_url="${file_server_url}/download/builds/pingcap/cdc/etcd-v3.5.15-linux-amd64.tar.gz" sync_diff_inspector_url="${file_server_url}/download/builds/pingcap/cdc/sync_diff_inspector_hash-a129f096_linux-amd64.tar.gz" jq_download_url="${file_server_url}/download/builds/pingcap/test/jq-1.6/jq-linux64" schema_registry_url="${file_server_url}/download/builds/pingcap/cdc/schema-registry.tar.gz" From d70369d79c9352c438aeb2385e02111e716e0ed5 Mon Sep 17 00:00:00 2001 From: nhsmw Date: Wed, 11 Sep 2024 15:40:14 +0800 Subject: [PATCH 31/37] test(ticdc): fix data inconsistence on integration_tests (#11584) close pingcap/tiflow#11583 --- tests/integration_tests/kafka_simple_basic/data/data.sql | 2 +- tests/integration_tests/kafka_simple_basic_avro/data/data.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration_tests/kafka_simple_basic/data/data.sql b/tests/integration_tests/kafka_simple_basic/data/data.sql index 71709dbafed..426c352b589 100644 --- a/tests/integration_tests/kafka_simple_basic/data/data.sql +++ b/tests/integration_tests/kafka_simple_basic/data/data.sql @@ -179,7 +179,7 @@ alter table tp_time add column c_timestamp2 timestamp default now(); insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) values ('2024-03-09', '2022-02-22 22:22:22', '2020-02-20 02:20:20', '02:20:20', '2021'); -delete from tp_time where id in (2, 3, 4); +delete from tp_time where id in (1, 2, 3, 4, 5); alter table tp_time alter column c_timestamp2 drop default; diff --git a/tests/integration_tests/kafka_simple_basic_avro/data/data.sql b/tests/integration_tests/kafka_simple_basic_avro/data/data.sql index d52461d6c92..8317cf4ee32 100644 --- a/tests/integration_tests/kafka_simple_basic_avro/data/data.sql +++ b/tests/integration_tests/kafka_simple_basic_avro/data/data.sql @@ -231,7 +231,7 @@ alter table tp_time add column c_timestamp2 timestamp default now(); insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) values ('2024-03-09', '2022-02-22 22:22:22', '2020-02-20 02:20:20', '02:20:20', '2021'); -delete from tp_time where id in (2, 3, 4); +delete from tp_time where id in (1, 2, 3, 4, 5); alter table tp_time alter column c_timestamp2 drop default; From c134bc06772b1e1939f074a1ad513d5a821b557c Mon Sep 17 00:00:00 2001 From: wk989898 Date: Wed, 11 Sep 2024 17:21:23 +0800 Subject: [PATCH 32/37] update --- cdc/sink/dmlsink/txn/mysql/dml_test.go | 31 +++++++++++++++++++ scripts/download-integration-test-binaries.sh | 2 +- 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/cdc/sink/dmlsink/txn/mysql/dml_test.go b/cdc/sink/dmlsink/txn/mysql/dml_test.go index fd2807a7510..ffc3c846982 100644 --- a/cdc/sink/dmlsink/txn/mysql/dml_test.go +++ b/cdc/sink/dmlsink/txn/mysql/dml_test.go @@ -250,6 +250,37 @@ func TestPrepareUpdate(t *testing.T) { expectedSQL: "UPDATE `test`.`t1` SET `a` = ?, `b` = ? WHERE `a` = ? AND `b` = ? LIMIT 1", expectedArgs: []interface{}{2, "世界", 1, "你好"}, }, + { + quoteTable: "`test`.`t1`", + preCols: []*model.Column{ + { + Name: "a", + Type: mysql.TypeLong, + Flag: model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }, + { + Name: "b", + Type: mysql.TypeTiDBVectorFloat32, + Value: util.Must(types.ParseVectorFloat32("[1.0,-2,0.33,-4.4,55]")), + }, + }, + cols: []*model.Column{ + { + Name: "a", + Type: mysql.TypeLong, + Flag: model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }, + { + Name: "b", + Type: mysql.TypeTiDBVectorFloat32, + Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), + }, + }, + expectedSQL: "UPDATE `test`.`t1` SET `a` = ?, `b` = ? WHERE `a` = ? LIMIT 1", + expectedArgs: []interface{}{1, "[1,2,3,4,5]", 1}, + }, } for _, tc := range testCases { query, args := prepareUpdate(tc.quoteTable, tc.preCols, tc.cols, false) diff --git a/scripts/download-integration-test-binaries.sh b/scripts/download-integration-test-binaries.sh index 80f20e27c68..fceeb049842 100755 --- a/scripts/download-integration-test-binaries.sh +++ b/scripts/download-integration-test-binaries.sh @@ -141,7 +141,7 @@ function download_binaries() { tiflash_download_url="${file_server_url}/download/builds/pingcap/tiflash/${tiflash_branch}/${tiflash_sha1}/centos7/tiflash.tar.gz" minio_download_url="${file_server_url}/download/minio.tar.gz" go_ycsb_download_url="${file_server_url}/download/builds/pingcap/go-ycsb/test-br/go-ycsb" - etcd_download_url="${file_server_url}/download/builds/pingcap/cdc/etcd-v3.5.15-linux-amd64.tar.gz" + etcd_download_url="${file_server_url}/download/builds/pingcap/cdc/etcd-v3.4.7-linux-amd64.tar.gz" sync_diff_inspector_url="${file_server_url}/download/builds/pingcap/cdc/sync_diff_inspector_hash-a129f096_linux-amd64.tar.gz" jq_download_url="${file_server_url}/download/builds/pingcap/test/jq-1.6/jq-linux64" schema_registry_url="${file_server_url}/download/builds/pingcap/cdc/schema-registry.tar.gz" From 4b3bb17fd29e0657a96533e5938cc0fc8c9f4fa8 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 12 Sep 2024 11:36:59 +0800 Subject: [PATCH 33/37] set HasVectorType default false --- cdc/sink/ddlsink/mysql/format_ddl_test.go | 22 +++++++++++++--------- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 1 + cdc/sink/dmlsink/txn/mysql/mysql_test.go | 9 +++++---- pkg/sink/mysql/config.go | 10 +++++++++- 4 files changed, 28 insertions(+), 14 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/format_ddl_test.go b/cdc/sink/ddlsink/mysql/format_ddl_test.go index 759af5e7656..fda65597a5d 100644 --- a/cdc/sink/ddlsink/mysql/format_ddl_test.go +++ b/cdc/sink/ddlsink/mysql/format_ddl_test.go @@ -17,27 +17,31 @@ import ( "bytes" "testing" - "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/format" "github.com/stretchr/testify/require" - "go.uber.org/zap" ) func TestFormatQuery(t *testing.T) { sql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` VECTOR(5));" - expectSQL := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT)" + expectSQL := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT);" p := parser.New() stmt, err := p.ParseOneStmt(sql, "", "") - if err != nil { - log.Error("format query parse one stmt failed", zap.Error(err)) - } + require.NoError(t, err) stmt.Accept(&visiter{}) buf := new(bytes.Buffer) restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, buf) - if err = stmt.Restore(restoreCtx); err != nil { - log.Error("format query restore failed", zap.Error(err)) - } + err = stmt.Restore(restoreCtx) + require.NoError(t, err) require.Equal(t, buf.String(), expectSQL) } + +func BenchmarkFormatQuery(b *testing.B) { + sql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT);" + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + formatQuery(sql) + } +} diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 9670831c774..8cb40fe4186 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -291,6 +291,7 @@ func needSwitchDB(ddl *model.DDLEvent) bool { // needFormatDDL checks vector type support func needFormatDDL(db *sql.DB, cfg *pmysql.Config) bool { if !cfg.HasVectorType { + log.Warn("please set `has-vector-type` to be true if data is vector", zap.Any("hasVectorType", cfg.HasVectorType)) return false } versionInfo, err := export.SelectVersion(db) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index ef0e6120f42..26ff9cc28d8 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -121,7 +121,8 @@ func TestPrepareDML(t *testing.T) { tableInfoVector := model.BuildTableInfo("common_1", "uk_without_pk", []*model.Column{ nil, { Name: "a1", - Type: mysql.TypeTiDBVectorFloat32, + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag | model.UniqueKeyFlag, }, { Name: "a3", Type: mysql.TypeTiDBVectorFloat32, @@ -208,8 +209,8 @@ func TestPrepareDML(t *testing.T) { []*model.Column{ nil, { Name: "a1", - Type: mysql.TypeTiDBVectorFloat32, - Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), + Type: mysql.TypeLong, + Value: 1, }, { Name: "a3", Type: mysql.TypeTiDBVectorFloat32, @@ -221,7 +222,7 @@ func TestPrepareDML(t *testing.T) { expected: &preparedDMLs{ startTs: []model.Ts{418658114257813518}, sqls: []string{"INSERT INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)"}, - values: [][]interface{}{{"[1,2,3,4,5]", "[1.1,-2,3.33,-4.12,-5]"}}, + values: [][]interface{}{{1, "[1.1,-2,3.33,-4.12,-5]"}}, rowCount: 1, approximateSize: 63, }, diff --git a/pkg/sink/mysql/config.go b/pkg/sink/mysql/config.go index 841d385b298..f23f892167e 100644 --- a/pkg/sink/mysql/config.go +++ b/pkg/sink/mysql/config.go @@ -78,7 +78,7 @@ const ( // defaultcachePrepStmts is the default value of cachePrepStmts defaultCachePrepStmts = true - defaultHasVectorType = true + defaultHasVectorType = false ) type urlConfig struct { @@ -98,6 +98,7 @@ type urlConfig struct { EnableBatchDML *bool `form:"batch-dml-enable"` EnableMultiStatement *bool `form:"multi-stmt-enable"` EnableCachePreparedStatement *bool `form:"cache-prep-stmts"` + HasVectorType *bool `form:"has-vector-type"` } // Config is the configs for MySQL backend. @@ -201,6 +202,7 @@ func (c *Config) Apply( } getBatchDMLEnable(urlParameter, &c.BatchDMLEnable) + getHasVectorType(urlParameter, &c.HasVectorType) getMultiStmtEnable(urlParameter, &c.MultiStmtEnable) getCachePrepStmts(urlParameter, &c.CachePrepStmts) c.ForceReplicate = replicaConfig.ForceReplicate @@ -450,6 +452,12 @@ func getBatchDMLEnable(values *urlConfig, batchDMLEnable *bool) { } } +func getHasVectorType(values *urlConfig, hasVectorType *bool) { + if values.HasVectorType != nil { + *hasVectorType = *values.HasVectorType + } +} + func getMultiStmtEnable(values *urlConfig, multiStmtEnable *bool) { if values.EnableMultiStatement != nil { *multiStmtEnable = *values.EnableMultiStatement From c157b20588977ed4eb9c7e13dcac2f336a769f16 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 12 Sep 2024 11:48:25 +0800 Subject: [PATCH 34/37] chore: update print log --- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 5 +++-- pkg/sink/mysql/config.go | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 8cb40fe4186..ed5be1b8158 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -291,7 +291,8 @@ func needSwitchDB(ddl *model.DDLEvent) bool { // needFormatDDL checks vector type support func needFormatDDL(db *sql.DB, cfg *pmysql.Config) bool { if !cfg.HasVectorType { - log.Warn("please set `has-vector-type` to be true if data is vector", zap.Any("hasVectorType", cfg.HasVectorType)) + log.Warn("please set `has-vector-type` to be true if a column is vector type when the downstream is not TiDB or TiDB version less than specify version", + zap.Any("hasVectorType", cfg.HasVectorType), zap.Any("supportVectorVersion", defaultSupportVectorVersion)) return false } versionInfo, err := export.SelectVersion(db) @@ -302,7 +303,7 @@ func needFormatDDL(db *sql.DB, cfg *pmysql.Config) bool { serverInfo := version.ParseServerInfo(versionInfo) version := semver.New(defaultSupportVectorVersion) if !cfg.IsTiDB || serverInfo.ServerVersion.LessThan(*version) { - log.Error("downstream unsupport vector type. we convert it to longtext", zap.String("version", serverInfo.ServerVersion.String()), zap.String("supportVersion", defaultSupportVectorVersion), zap.Bool("isTiDB", cfg.IsTiDB)) + log.Error("downstream unsupport vector type. it will be converted to longtext", zap.String("version", serverInfo.ServerVersion.String()), zap.String("supportVectorVersion", defaultSupportVectorVersion), zap.Bool("isTiDB", cfg.IsTiDB)) return true } return false diff --git a/pkg/sink/mysql/config.go b/pkg/sink/mysql/config.go index f23f892167e..9e9f149eee3 100644 --- a/pkg/sink/mysql/config.go +++ b/pkg/sink/mysql/config.go @@ -120,7 +120,7 @@ type Config struct { // IsBDRModeSupported is true if the downstream is TiDB and write source is existed. // write source exists when the downstream is TiDB and version is greater than or equal to v6.5.0. IsWriteSourceExisted bool - HasVectorType bool // HasVectorType is true if the data is vector + HasVectorType bool // HasVectorType is true if the column is vector type SourceID uint64 BatchDMLEnable bool From d97adf59b8844927e64b8059d4056326396fe49e Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 12 Sep 2024 12:35:08 +0800 Subject: [PATCH 35/37] fix test --- cdc/sink/ddlsink/mysql/format_ddl_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/format_ddl_test.go b/cdc/sink/ddlsink/mysql/format_ddl_test.go index fda65597a5d..2059877ce79 100644 --- a/cdc/sink/ddlsink/mysql/format_ddl_test.go +++ b/cdc/sink/ddlsink/mysql/format_ddl_test.go @@ -23,8 +23,8 @@ import ( ) func TestFormatQuery(t *testing.T) { - sql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` VECTOR(5));" - expectSQL := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT);" + sql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` VECTOR(5))" + expectSQL := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT)" p := parser.New() stmt, err := p.ParseOneStmt(sql, "", "") require.NoError(t, err) @@ -38,7 +38,7 @@ func TestFormatQuery(t *testing.T) { } func BenchmarkFormatQuery(b *testing.B) { - sql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT);" + sql := "CREATE TABLE `test` (`id` INT PRIMARY KEY,`data` LONGTEXT)" b.ReportAllocs() b.ResetTimer() for i := 0; i < b.N; i++ { From 15ddca35abe40118a4d223f2bfcb74cda3cc42af Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 20 Sep 2024 14:29:25 +0800 Subject: [PATCH 36/37] Revert "sink(ticdc): use admin statement to query async ddl status (#11535)" This reverts commit 053633232faa8d09538bfc5d35537e6ab9b00635. --- cdc/sink/ddlsink/mysql/async_ddl.go | 78 ++++++++---------------- cdc/sink/ddlsink/mysql/async_ddl_test.go | 28 +-------- 2 files changed, 27 insertions(+), 79 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/async_ddl.go b/cdc/sink/ddlsink/mysql/async_ddl.go index 8e480bb06b0..1692a394f37 100644 --- a/cdc/sink/ddlsink/mysql/async_ddl.go +++ b/cdc/sink/ddlsink/mysql/async_ddl.go @@ -15,29 +15,25 @@ package mysql import ( "context" + "database/sql" "fmt" "time" "github.com/pingcap/log" - "github.com/pingcap/tidb/dumpling/export" timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) -const timeout = 5 * time.Second - -// TODO: Use the flollowing SQL to check the ddl job status after tidb optimize -// the information_schema.ddl_jobs table. Ref: https://github.com/pingcap/tidb/issues/55725 -// -// SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY -// FROM information_schema.ddl_jobs var checkRunningAddIndexSQL = ` -ADMIN SHOW DDL JOBS 1 +SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY +FROM information_schema.ddl_jobs WHERE DB_NAME = "%s" AND TABLE_NAME = "%s" AND JOB_TYPE LIKE "add index%%" - AND (STATE = "running" OR STATE = "queueing"); + AND (STATE = "running" OR STATE = "queueing") +LIMIT 1; ` func (m *DDLSink) shouldAsyncExecDDL(ddl *model.DDLEvent) bool { @@ -96,23 +92,9 @@ func (m *DDLSink) asyncExecDDL(ctx context.Context, ddl *model.DDLEvent) error { } } -func (m *DDLSink) needWaitAsyncExecDone(t timodel.ActionType) bool { - if !m.cfg.IsTiDB { - return false - } - switch t { - case timodel.ActionCreateTable, timodel.ActionCreateTables: - return false - case timodel.ActionCreateSchema: - return false - default: - return true - } -} - // Should always wait for async ddl done before executing the next ddl. func (m *DDLSink) waitAsynExecDone(ctx context.Context, ddl *model.DDLEvent) { - if !m.needWaitAsyncExecDone(ddl.Type) { + if !m.cfg.IsTiDB { return } @@ -123,6 +105,9 @@ func (m *DDLSink) waitAsynExecDone(ctx context.Context, ddl *model.DDLEvent) { if ddl.PreTableInfo != nil { tables[ddl.PreTableInfo.TableName] = struct{}{} } + if len(tables) == 0 || m.checkAsyncExecDDLDone(ctx, tables) { + return + } log.Debug("wait async exec ddl done", zap.String("namespace", m.id.Namespace), @@ -130,10 +115,6 @@ func (m *DDLSink) waitAsynExecDone(ctx context.Context, ddl *model.DDLEvent) { zap.Any("tables", tables), zap.Uint64("commitTs", ddl.CommitTs), zap.String("ddl", ddl.Query)) - if len(tables) == 0 || m.checkAsyncExecDDLDone(ctx, tables) { - return - } - ticker := time.NewTicker(5 * time.Second) defer ticker.Stop() for { @@ -150,8 +131,6 @@ func (m *DDLSink) waitAsynExecDone(ctx context.Context, ddl *model.DDLEvent) { } func (m *DDLSink) checkAsyncExecDDLDone(ctx context.Context, tables map[model.TableName]struct{}) bool { - ctx, cancel := context.WithTimeout(ctx, timeout) - defer cancel() for table := range tables { done := m.doCheck(ctx, table) if !done { @@ -162,7 +141,6 @@ func (m *DDLSink) checkAsyncExecDDLDone(ctx context.Context, tables map[model.Ta } func (m *DDLSink) doCheck(ctx context.Context, table model.TableName) (done bool) { - start := time.Now() if v, ok := m.lastExecutedNormalDDLCache.Get(table); ok { ddlType := v.(timodel.ActionType) if ddlType == timodel.ActionAddIndex { @@ -174,41 +152,35 @@ func (m *DDLSink) doCheck(ctx context.Context, table model.TableName) (done bool return true } - rows, err := m.db.QueryContext(ctx, fmt.Sprintf(checkRunningAddIndexSQL, table.Schema, table.Table)) - defer func() { - if rows != nil { - _ = rows.Err() - } - }() - if err != nil { + ret := m.db.QueryRowContext(ctx, fmt.Sprintf(checkRunningAddIndexSQL, table.Schema, table.Table)) + if ret.Err() != nil { log.Error("check async exec ddl failed", zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID), - zap.Error(err)) + zap.Error(ret.Err())) return true } - rets, err := export.GetSpecifiedColumnValuesAndClose(rows, "JOB_ID", "JOB_TYPE", "SCHEMA_STATE", "STATE") - if err != nil { - log.Error("check async exec ddl failed", - zap.String("namespace", m.id.Namespace), - zap.String("changefeed", m.id.ID), - zap.Error(err)) + var jobID, jobType, schemaState, schemaID, tableID, state, query string + if err := ret.Scan(&jobID, &jobType, &schemaState, &schemaID, &tableID, &state, &query); err != nil { + if !errors.Is(err, sql.ErrNoRows) { + log.Error("check async exec ddl failed", + zap.String("namespace", m.id.Namespace), + zap.String("changefeed", m.id.ID), + zap.Error(err)) + } return true } - if len(rets) == 0 { - return true - } - ret := rets[0] - jobID, jobType, schemaState, state := ret[0], ret[1], ret[2], ret[3] log.Info("async ddl is still running", zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID), - zap.Duration("checkDuration", time.Since(start)), zap.String("table", table.String()), zap.String("jobID", jobID), zap.String("jobType", jobType), zap.String("schemaState", schemaState), - zap.String("state", state)) + zap.String("schemaID", schemaID), + zap.String("tableID", tableID), + zap.String("state", state), + zap.String("query", query)) return false } diff --git a/cdc/sink/ddlsink/mysql/async_ddl_test.go b/cdc/sink/ddlsink/mysql/async_ddl_test.go index b05f09bb979..871080c2eb5 100644 --- a/cdc/sink/ddlsink/mysql/async_ddl_test.go +++ b/cdc/sink/ddlsink/mysql/async_ddl_test.go @@ -45,14 +45,8 @@ func TestWaitAsynExecDone(t *testing.T) { // Case 1: there is a running add index job mock.ExpectQuery(fmt.Sprintf(checkRunningAddIndexSQL, "test", "sbtest0")).WillReturnRows( - sqlmock.NewRows([]string{ - "JOB_ID", "DB_NAME", "TABLE_NAME", "JOB_TYPE", "SCHEMA_STATE", "SCHEMA_ID", "TABLE_ID", - "ROW_COUNT", "CREATE_TIME", "START_TIME", "END_TIME", "STATE", - }).AddRow( - 1, "test", "sbtest0", "add index", "write reorganization", 1, 1, 0, time.Now(), nil, time.Now(), "running", - ).AddRow( - 2, "test", "sbtest0", "add index", "write reorganization", 1, 1, 0, time.Now(), time.Now(), time.Now(), "queueing", - ), + sqlmock.NewRows([]string{"JOB_ID", "JOB_TYPE", "SCHEMA_STATE", "SCHEMA_ID", "TABLE_ID", "STATE", "QUERY"}). + AddRow("1", "add index", "running", "1", "1", "running", "Create index idx1 on test.sbtest0(a)"), ) // Case 2: there is no running add index job // Case 3: no permission to query ddl_jobs, TiDB will return empty result @@ -163,21 +157,3 @@ func TestAsyncExecAddIndex(t *testing.T) { require.True(t, time.Since(start) >= 10*time.Second) sink.Close() } - -func TestNeedWaitAsyncExecDone(t *testing.T) { - sink := &DDLSink{ - cfg: &pmysql.Config{ - IsTiDB: false, - }, - } - require.False(t, sink.needWaitAsyncExecDone(timodel.ActionTruncateTable)) - - sink.cfg.IsTiDB = true - require.True(t, sink.needWaitAsyncExecDone(timodel.ActionTruncateTable)) - require.True(t, sink.needWaitAsyncExecDone(timodel.ActionDropTable)) - require.True(t, sink.needWaitAsyncExecDone(timodel.ActionDropIndex)) - - require.False(t, sink.needWaitAsyncExecDone(timodel.ActionCreateTable)) - require.False(t, sink.needWaitAsyncExecDone(timodel.ActionCreateTables)) - require.False(t, sink.needWaitAsyncExecDone(timodel.ActionCreateSchema)) -} From 8fb9173a5bb6d8d14c057f6a0717779128a9bd60 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Fri, 20 Sep 2024 14:31:26 +0800 Subject: [PATCH 37/37] remove unused code --- cdc/sink/util/vector.go | 87 ----------------------------------------- 1 file changed, 87 deletions(-) delete mode 100644 cdc/sink/util/vector.go diff --git a/cdc/sink/util/vector.go b/cdc/sink/util/vector.go deleted file mode 100644 index 1f6fa6922b1..00000000000 --- a/cdc/sink/util/vector.go +++ /dev/null @@ -1,87 +0,0 @@ -// Copyright 2024 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "encoding/json" - "reflect" - "strconv" - "unsafe" - - "github.com/pingcap/log" - "github.com/pingcap/tidb/pkg/types" - "go.uber.org/zap" -) - -// CheckVectorIndexForColumnModify checks Vector Index constraints for MODIFY COLUMN. -// func CheckVectorIndexForColumnModify(oldCol *table.Column, newCol *table.Column) error { -// if oldCol.VectorIndex == nil && newCol.VectorIndex == nil { -// return nil -// } -// if oldCol.VectorIndex == nil && newCol.VectorIndex != nil { -// return errors.Errorf("currently HNSW index can be only defined when creating the table") -// } -// if oldCol.VectorIndex != nil && newCol.VectorIndex == nil { -// return errors.Errorf("currently HNSW index can not be removed") -// } -// if oldCol.FieldType.GetFlen() != newCol.FieldType.GetFlen() { -// return errors.New("cannot modify vector column's dimension when HNSW index is defined") -// } -// if oldCol.FieldType.GetType() != newCol.FieldType.GetType() { -// return errors.New("cannot modify column data type when HNSW index is defined") -// } -// if *(oldCol.VectorIndex) != *(newCol.VectorIndex) { -// return errors.New("currently HNSW index cannot be modified") -// } -// return nil -// } - -// ParseVectorFromElement parse element to VectorFloat32 -func ParseVectorFromElement(values []float32) (types.VectorFloat32, error) { - dim := len(values) - if err := types.CheckVectorDimValid(dim); err != nil { - return types.ZeroVectorFloat32, err - } - vec := types.InitVectorFloat32(dim) - copy(vec.Elements(), values) - return vec, nil -} - -// VectorElement2String parse elements to string -func VectorElement2String(elements []interface{}) string { - buf := make([]byte, 0, 2+len(elements)*2) - buf = append(buf, '[') - for i, val := range elements { - if i > 0 { - buf = append(buf, ',') - } - switch v := val.(type) { - case json.Number: - num, err := v.Float64() - if err != nil { - log.Panic("failed to decode val", zap.Any("val", val), zap.Error(err)) - } - buf = strconv.AppendFloat(buf, num, 'f', -1, 32) - case float32: - buf = strconv.AppendFloat(buf, float64(v), 'f', -1, 32) - case float64: - buf = strconv.AppendFloat(buf, v, 'f', -1, 32) - default: - log.Panic("failed to decode val type", zap.Any("val", val), zap.Any("type", reflect.TypeOf(v))) - } - } - buf = append(buf, ']') - // buf is not used elsewhere, so it's safe to just cast to String - return unsafe.String(unsafe.SliceData(buf), len(buf)) -}