diff --git a/cdc/model/schema_storage.go b/cdc/model/schema_storage.go index dad989ad89f..bc1e45946ef 100644 --- a/cdc/model/schema_storage.go +++ b/cdc/model/schema_storage.go @@ -212,6 +212,9 @@ func (ti *TableInfo) initColumnsFlag() { if mysql.HasUnsignedFlag(colInfo.GetFlag()) { flag.SetIsUnsigned() } + if mysql.HasZerofillFlag(colInfo.GetFlag()) { + flag.SetZeroFill() + } ti.ColumnsFlag[colInfo.ID] = flag } diff --git a/cdc/model/sink.go b/cdc/model/sink.go index adf063b1bca..3615a8a867d 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -76,8 +76,20 @@ const ( NullableFlag // UnsignedFlag means the column stores an unsigned integer UnsignedFlag + // ZerofillFlag means the column is zerofill + ZerofillFlag ) +// SetZeroFill sets ZerofillFlag +func (b *ColumnFlagType) SetZeroFill() { + (*util.Flag)(b).Add(util.Flag(ZerofillFlag)) +} + +// IsZerofill shows whether ZerofillFlag is set +func (b *ColumnFlagType) IsZerofill() bool { + return (*util.Flag)(b).HasAll(util.Flag(ZerofillFlag)) +} + // SetIsBinary sets BinaryFlag func (b *ColumnFlagType) SetIsBinary() { (*util.Flag)(b).Add(util.Flag(BinaryFlag)) diff --git a/cdc/sink/codec/avro/avro_test.go b/cdc/sink/codec/avro/avro_test.go index 5d0d29c1969..1af71181a9f 100644 --- a/cdc/sink/codec/avro/avro_test.go +++ b/cdc/sink/codec/avro/avro_test.go @@ -71,22 +71,6 @@ func teardownEncoderAndSchemaRegistry() { stopHTTPInterceptForTestingRegistry() } -func setBinChsClnFlag(ft *types.FieldType) *types.FieldType { - types.SetBinChsClnFlag(ft) - return ft -} - -//nolint:unparam -func setFlag(ft *types.FieldType, flag uint) *types.FieldType { - ft.SetFlag(flag) - return ft -} - -func setElems(ft *types.FieldType, elems []string) *types.FieldType { - ft.SetElems(elems) - return ft -} - type avroTestColumnTuple struct { col model.Column colInfo rowcodec.ColInfo @@ -162,7 +146,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 6, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeTiny), uint(model.UnsignedFlag)), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeTiny)), }, avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, int32(1), "int", @@ -178,7 +162,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 7, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeShort), uint(model.UnsignedFlag)), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeShort)), }, avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, int32(1), "int", @@ -194,7 +178,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 8, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeInt24), uint(model.UnsignedFlag)), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeInt24)), }, avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, int32(1), "int", @@ -210,7 +194,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 9, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeLong), uint(model.UnsignedFlag)), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeLong)), }, avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, int64(1), "long", @@ -226,10 +210,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 10, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag( - types.NewFieldType(mysql.TypeLonglong), - uint(model.UnsignedFlag), - ), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeLonglong)), }, avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "BIGINT UNSIGNED"}}, int64(1), "long", @@ -374,7 +355,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 22, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -390,7 +371,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 23, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -406,7 +387,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 24, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -422,7 +403,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 25, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -438,7 +419,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 26, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -454,7 +435,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 27, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -470,7 +451,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 28, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeString)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeString)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -481,7 +462,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 29, IsPKHandle: false, VirtualGenCol: false, - Ft: setElems(types.NewFieldType(mysql.TypeEnum), []string{"a,", "b"}), + Ft: common.SetElems(types.NewFieldType(mysql.TypeEnum), []string{"a,", "b"}), }, avroSchema{ Type: "string", @@ -495,7 +476,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 30, IsPKHandle: false, VirtualGenCol: false, - Ft: setElems(types.NewFieldType(mysql.TypeSet), []string{"a,", "b"}), + Ft: common.SetElems(types.NewFieldType(mysql.TypeSet), []string{"a,", "b"}), }, avroSchema{ Type: "string", diff --git a/cdc/sink/codec/canal/canal_encoder.go b/cdc/sink/codec/canal/canal_encoder.go index 19350a473d6..d09dcc11a80 100644 --- a/cdc/sink/codec/canal/canal_encoder.go +++ b/cdc/sink/codec/canal/canal_encoder.go @@ -162,7 +162,7 @@ func newBatchEncoder(config *common.Config) codec.EventBatchEncoder { encoder := &BatchEncoder{ messages: &canal.Messages{}, callbackBuf: make([]func(), 0), - entryBuilder: newCanalEntryBuilder(), + entryBuilder: newCanalEntryBuilder(config), config: config, } diff --git a/cdc/sink/codec/canal/canal_encoder_test.go b/cdc/sink/codec/canal/canal_encoder_test.go index b07cee9eeb0..69378809073 100644 --- a/cdc/sink/codec/canal/canal_encoder_test.go +++ b/cdc/sink/codec/canal/canal_encoder_test.go @@ -19,6 +19,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec/common" "github.com/pingcap/tiflow/pkg/config" @@ -26,14 +27,87 @@ import ( "github.com/stretchr/testify/require" ) +var ( + rowCases = [][]*model.RowChangedEvent{ + {{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + Columns: []*model.Column{{ + Name: "col1", + Type: mysql.TypeVarchar, + Value: []byte("aa"), + }}, + }}, + { + { + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + Columns: []*model.Column{{ + Name: "col1", + Type: mysql.TypeVarchar, + Value: []byte("aa"), + }}, + }, + { + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + }, + }, + } + + ddlCases = [][]*model.DDLEvent{ + {{ + CommitTs: 1, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", Table: "b", + }, + }, + Query: "create table a", + Type: 1, + }}, + { + { + CommitTs: 2, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", Table: "b", + }, + }, + Query: "create table b", + Type: 3, + }, + { + CommitTs: 3, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", Table: "b", + }, + }, + Query: "create table c", + Type: 3, + }, + }, + } +) + func TestCanalBatchEncoder(t *testing.T) { - t.Parallel() - s := defaultCanalBatchTester - for _, cs := range s.rowCases { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(10) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + for _, cs := range rowCases { encoder := newBatchEncoder(common.NewConfig(config.ProtocolCanal)) for _, row := range cs { + _, _, colInfo := tableInfo.GetRowColInfos() + row.TableInfo = tableInfo + row.ColInfos = colInfo err := encoder.AppendRowChangedEvent(context.Background(), "", row, nil) - require.Nil(t, err) + require.NoError(t, err) } res := encoder.Build() @@ -41,7 +115,6 @@ func TestCanalBatchEncoder(t *testing.T) { require.Nil(t, res) continue } - require.Len(t, res, 1) require.Nil(t, res[0].Key) require.Equal(t, len(cs), res[0].GetRowsCount()) @@ -56,33 +129,36 @@ func TestCanalBatchEncoder(t *testing.T) { require.Equal(t, len(cs), len(messages.GetMessages())) } - for _, cs := range s.ddlCases { + for _, cs := range ddlCases { encoder := newBatchEncoder(common.NewConfig(config.ProtocolCanal)) for _, ddl := range cs { msg, err := encoder.EncodeDDLEvent(ddl) - require.Nil(t, err) + require.NoError(t, err) require.NotNil(t, msg) require.Nil(t, msg.Key) packet := &canal.Packet{} err = proto.Unmarshal(msg.Value, packet) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, canal.PacketType_MESSAGES, packet.GetType()) messages := &canal.Messages{} err = proto.Unmarshal(packet.GetBody(), messages) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 1, len(messages.GetMessages())) - require.Nil(t, err) + require.NoError(t, err) } } } func TestCanalAppendRowChangedEventWithCallback(t *testing.T) { - encoder := newBatchEncoder(common.NewConfig(config.ProtocolCanal)) - require.NotNil(t, encoder) + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() - count := 0 + sql := `create table test.t(a varchar(10) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() row := &model.RowChangedEvent{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, @@ -91,8 +167,15 @@ func TestCanalAppendRowChangedEventWithCallback(t *testing.T) { Type: mysql.TypeVarchar, Value: []byte("aa"), }}, + TableInfo: tableInfo, + ColInfos: colInfo, } + encoder := newBatchEncoder(common.NewConfig(config.ProtocolCanal)) + require.NotNil(t, encoder) + + count := 0 + tests := []struct { row *model.RowChangedEvent callback func() diff --git a/cdc/sink/codec/canal/canal_entry.go b/cdc/sink/codec/canal/canal_entry.go index 705e12f0dd6..faac0f2401c 100644 --- a/cdc/sink/codec/canal/canal_entry.go +++ b/cdc/sink/codec/canal/canal_entry.go @@ -18,14 +18,14 @@ import ( "math" "reflect" "strconv" - "strings" "github.com/golang/protobuf/proto" // nolint:staticcheck "github.com/pingcap/errors" mm "github.com/pingcap/tidb/parser/model" + timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/types" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec/common" "github.com/pingcap/tiflow/cdc/sink/codec/internal" cerror "github.com/pingcap/tiflow/pkg/errors" canal "github.com/pingcap/tiflow/proto/canal" @@ -43,12 +43,14 @@ const ( type canalEntryBuilder struct { bytesDecoder *encoding.Decoder // default charset is ISO-8859-1 + config *common.Config } // newCanalEntryBuilder creates a new canalEntryBuilder -func newCanalEntryBuilder() *canalEntryBuilder { +func newCanalEntryBuilder(config *common.Config) *canalEntryBuilder { return &canalEntryBuilder{ bytesDecoder: charmap.ISO8859_1.NewDecoder(), + config: config, } } @@ -78,7 +80,7 @@ func (b *canalEntryBuilder) buildHeader(commitTs uint64, schema string, table st // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/dbsync/src/main/java/com/taobao/tddl/dbsync/binlog/event/RowsLogBuffer.java#L276-L1147 // all value will be represented in string type // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L760-L855 -func (b *canalEntryBuilder) formatValue(value interface{}, javaType internal.JavaSQLType) (result string, err error) { +func (b *canalEntryBuilder) formatValue(value interface{}, isBinary bool) (result string, err error) { // value would be nil, if no value insert for the column. if value == nil { return "", nil @@ -96,20 +98,15 @@ func (b *canalEntryBuilder) formatValue(value interface{}, javaType internal.Jav case string: result = v case []byte: - // JavaSQLTypeVARCHAR / JavaSQLTypeCHAR / JavaSQLTypeBLOB / JavaSQLTypeCLOB / - // special handle for text and blob // see https://github.com/alibaba/canal/blob/9f6021cf36f78cc8ac853dcf37a1769f359b868b/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L801 - switch javaType { - // for normal text - case internal.JavaSQLTypeVARCHAR, internal.JavaSQLTypeCHAR, internal.JavaSQLTypeCLOB: - result = string(v) - default: - // JavaSQLTypeBLOB + if isBinary { decoded, err := b.bytesDecoder.Bytes(v) if err != nil { return "", err } result = string(decoded) + } else { + result = string(v) } default: result = fmt.Sprintf("%v", v) @@ -119,21 +116,21 @@ func (b *canalEntryBuilder) formatValue(value interface{}, javaType internal.Jav // build the Column in the canal RowData // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L756-L872 -func (b *canalEntryBuilder) buildColumn(c *model.Column, colName string, updated bool) (*canal.Column, error) { - mysqlType := getMySQLType(c.Type, c.Flag) +func (b *canalEntryBuilder) buildColumn(c *model.Column, columnInfo *timodel.ColumnInfo, updated bool) (*canal.Column, error) { + mysqlType := common.GetMySQLType(columnInfo, b.config.ContentCompatible) javaType, err := getJavaSQLType(c.Value, c.Type, c.Flag) if err != nil { return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) } - value, err := b.formatValue(c.Value, javaType) + value, err := b.formatValue(c.Value, c.Flag.IsBinary()) if err != nil { return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) } canalColumn := &canal.Column{ SqlType: int32(javaType), - Name: colName, + Name: c.Name, IsKey: c.Flag.IsPrimaryKey(), Updated: updated, IsNullPresent: &canal.Column_IsNull{IsNull: c.Value == nil}, @@ -146,11 +143,16 @@ func (b *canalEntryBuilder) buildColumn(c *model.Column, colName string, updated // build the RowData of a canal entry func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent, onlyHandleKeyColumns bool) (*canal.RowData, error) { var columns []*canal.Column - for _, column := range e.Columns { + for idx, column := range e.Columns { if column == nil { continue } - c, err := b.buildColumn(column, column.Name, !e.IsDelete()) + columnInfo, ok := e.TableInfo.GetColumnInfo(e.ColInfos[idx].ID) + if !ok { + return nil, cerror.ErrCanalEncodeFailed.GenWithStack( + "column info not found for column id: %d", e.ColInfos[idx].ID) + } + c, err := b.buildColumn(column, columnInfo, !e.IsDelete()) if err != nil { return nil, errors.Trace(err) } @@ -159,14 +161,19 @@ func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent, onlyHandleKey onlyHandleKeyColumns = onlyHandleKeyColumns && e.IsDelete() var preColumns []*canal.Column - for _, column := range e.PreColumns { + for idx, column := range e.PreColumns { if column == nil { continue } if onlyHandleKeyColumns && !column.Flag.IsHandleKey() { continue } - c, err := b.buildColumn(column, column.Name, !e.IsDelete()) + columnInfo, ok := e.TableInfo.GetColumnInfo(e.ColInfos[idx].ID) + if !ok { + return nil, cerror.ErrCanalEncodeFailed.GenWithStack( + "column info not found for column id: %d", e.ColInfos[idx].ID) + } + c, err := b.buildColumn(column, columnInfo, !e.IsDelete()) if err != nil { return nil, errors.Trace(err) } @@ -301,15 +308,6 @@ func isCanalDDL(t canal.EventType) bool { func getJavaSQLType(value interface{}, tp byte, flag model.ColumnFlagType) (result internal.JavaSQLType, err error) { javaType := internal.MySQLType2JavaType(tp, flag.IsBinary()) - - switch javaType { - case internal.JavaSQLTypeBINARY, internal.JavaSQLTypeVARBINARY, internal.JavaSQLTypeLONGVARBINARY: - if flag.IsBinary() { - return internal.JavaSQLTypeBLOB, nil - } - return internal.JavaSQLTypeCLOB, nil - } - // flag `isUnsigned` only for `numerical` and `bit`, `year` data type. if !flag.IsUnsigned() { return javaType, nil @@ -372,37 +370,3 @@ func getJavaSQLType(value interface{}, tp byte, flag model.ColumnFlagType) (resu return javaType, nil } - -// when encoding the canal format, for unsigned mysql type, add `unsigned` keyword. -// it should have the form `t unsigned`, such as `int unsigned` -func withUnsigned4MySQLType(mysqlType string, unsigned bool) string { - if unsigned && mysqlType != "bit" && mysqlType != "year" { - return mysqlType + " unsigned" - } - return mysqlType -} - -// when decoding the canal format, remove `unsigned` to get the original `mysql type`. -func trimUnsignedFromMySQLType(mysqlType string) string { - return strings.TrimSuffix(mysqlType, " unsigned") -} - -func getMySQLType(tp byte, flag model.ColumnFlagType) string { - mysqlType := types.TypeStr(tp) - // make `mysqlType` representation keep the same as the canal official implementation - mysqlType = withUnsigned4MySQLType(mysqlType, flag.IsUnsigned()) - - if !flag.IsBinary() { - return mysqlType - } - - if types.IsTypeBlob(tp) { - return strings.Replace(mysqlType, "text", "blob", 1) - } - - if types.IsTypeChar(tp) { - return strings.Replace(mysqlType, "char", "binary", 1) - } - - return mysqlType -} diff --git a/cdc/sink/codec/canal/canal_entry_test.go b/cdc/sink/codec/canal/canal_entry_test.go index a58361e1bde..d0fef0161cf 100644 --- a/cdc/sink/codec/canal/canal_entry_test.go +++ b/cdc/sink/codec/canal/canal_entry_test.go @@ -19,77 +19,70 @@ import ( "github.com/golang/protobuf/proto" mm "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec/common" "github.com/pingcap/tiflow/cdc/sink/codec/internal" + "github.com/pingcap/tiflow/pkg/config" canal "github.com/pingcap/tiflow/proto/canal" "github.com/stretchr/testify/require" "golang.org/x/text/encoding/charmap" ) -func TestGetMySQLTypeAndJavaSQLType(t *testing.T) { - t.Parallel() - canalEntryBuilder := newCanalEntryBuilder() - for _, item := range testColumnsTable { - obtainedMySQLType := getMySQLType(item.column.Type, item.column.Flag) - require.Equal(t, item.expectedMySQLType, obtainedMySQLType) +func TestInsert(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() - obtainedJavaSQLType, err := getJavaSQLType(item.column.Value, item.column.Type, item.column.Flag) - require.Nil(t, err) - require.Equal(t, item.expectedJavaSQLType, obtainedJavaSQLType) + sql := `create table test.t( + id int primary key, + name varchar(32), + tiny tinyint unsigned, + comment text, + bb blob)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) - if !item.column.Flag.IsBinary() { - obtainedFinalValue, err := canalEntryBuilder.formatValue(item.column.Value, obtainedJavaSQLType) - require.Nil(t, err) - require.Equal(t, item.expectedEncodedValue, obtainedFinalValue) - } - } -} - -func TestConvertEntry(t *testing.T) { - t.Parallel() - testInsert(t) - testUpdate(t) - testDelete(t) - testDdl(t) -} + _, _, colInfos := tableInfo.GetRowColInfos() -func testInsert(t *testing.T) { - testCaseInsert := &model.RowChangedEvent{ + event := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ Schema: "cdc", Table: "person", }, + TableInfo: tableInfo, Columns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, {Name: "name", Type: mysql.TypeVarchar, Value: "Bob"}, {Name: "tiny", Type: mysql.TypeTiny, Value: 255}, {Name: "comment", Type: mysql.TypeBlob, Value: []byte("测试")}, - {Name: "blob", Type: mysql.TypeBlob, Value: []byte("测试blob"), Flag: model.BinaryFlag}, + {Name: "bb", Type: mysql.TypeBlob, Value: []byte("测试blob"), Flag: model.BinaryFlag}, }, + ColInfos: colInfos, } - builder := newCanalEntryBuilder() - entry, err := builder.fromRowEvent(testCaseInsert, false) - require.Nil(t, err) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := newCanalEntryBuilder(codecConfig) + entry, err := builder.fromRowEvent(event, false) + require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() require.Equal(t, int64(1591943372224), header.GetExecuteTime()) require.Equal(t, canal.Type_MYSQL, header.GetSourceType()) - require.Equal(t, testCaseInsert.Table.Schema, header.GetSchemaName()) - require.Equal(t, testCaseInsert.Table.Table, header.GetTableName()) + require.Equal(t, event.Table.Schema, header.GetSchemaName()) + require.Equal(t, event.Table.Table, header.GetTableName()) require.Equal(t, canal.EventType_INSERT, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) rc := &canal.RowChange{} err = proto.Unmarshal(store, rc) - require.Nil(t, err) + require.NoError(t, err) require.False(t, rc.GetIsDdl()) rowDatas := rc.GetRowDatas() require.Equal(t, 1, len(rowDatas)) columns := rowDatas[0].AfterColumns - require.Equal(t, len(testCaseInsert.Columns), len(columns)) + require.Equal(t, len(event.Columns), len(columns)) for _, col := range columns { require.True(t, col.GetUpdated()) switch col.GetName() { @@ -114,28 +107,38 @@ func testInsert(t *testing.T) { require.Equal(t, int32(internal.JavaSQLTypeCLOB), col.GetSqlType()) require.False(t, col.GetIsKey()) require.False(t, col.GetIsNull()) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, "测试", col.GetValue()) require.Equal(t, "text", col.GetMysqlType()) - case "blob": + case "bb": require.Equal(t, int32(internal.JavaSQLTypeBLOB), col.GetSqlType()) require.False(t, col.GetIsKey()) require.False(t, col.GetIsNull()) s, err := charmap.ISO8859_1.NewEncoder().String(col.GetValue()) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, "测试blob", s) require.Equal(t, "blob", col.GetMysqlType()) } } } -func testUpdate(t *testing.T) { - testCaseUpdate := &model.RowChangedEvent{ +func TestUpdate(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(id int primary key, name varchar(32))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + event := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ - Schema: "cdc", - Table: "person", + Schema: "test", + Table: "t", }, + TableInfo: tableInfo, Columns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, {Name: "name", Type: mysql.TypeVarchar, Value: "Bob"}, @@ -144,29 +147,31 @@ func testUpdate(t *testing.T) { {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 2}, {Name: "name", Type: mysql.TypeVarchar, Value: "Nancy"}, }, + ColInfos: colInfos, } - builder := newCanalEntryBuilder() - entry, err := builder.fromRowEvent(testCaseUpdate, false) - require.Nil(t, err) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := newCanalEntryBuilder(codecConfig) + entry, err := builder.fromRowEvent(event, false) + require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() require.Equal(t, int64(1591943372224), header.GetExecuteTime()) require.Equal(t, canal.Type_MYSQL, header.GetSourceType()) - require.Equal(t, testCaseUpdate.Table.Schema, header.GetSchemaName()) - require.Equal(t, testCaseUpdate.Table.Table, header.GetTableName()) + require.Equal(t, event.Table.Schema, header.GetSchemaName()) + require.Equal(t, event.Table.Table, header.GetTableName()) require.Equal(t, canal.EventType_UPDATE, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) rc := &canal.RowChange{} err = proto.Unmarshal(store, rc) - require.Nil(t, err) + require.NoError(t, err) require.False(t, rc.GetIsDdl()) rowDatas := rc.GetRowDatas() require.Equal(t, 1, len(rowDatas)) beforeColumns := rowDatas[0].BeforeColumns - require.Equal(t, len(testCaseUpdate.PreColumns), len(beforeColumns)) + require.Equal(t, len(event.PreColumns), len(beforeColumns)) for _, col := range beforeColumns { require.True(t, col.GetUpdated()) switch col.GetName() { @@ -186,7 +191,7 @@ func testUpdate(t *testing.T) { } afterColumns := rowDatas[0].AfterColumns - require.Equal(t, len(testCaseUpdate.Columns), len(afterColumns)) + require.Equal(t, len(event.Columns), len(afterColumns)) for _, col := range afterColumns { require.True(t, col.GetUpdated()) switch col.GetName() { @@ -206,37 +211,48 @@ func testUpdate(t *testing.T) { } } -func testDelete(t *testing.T) { - testCaseDelete := &model.RowChangedEvent{ +func TestDelete(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(id int primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + event := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ - Schema: "cdc", - Table: "person", + Schema: "test", + Table: "t", }, + TableInfo: tableInfo, PreColumns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, }, + ColInfos: colInfos, } - - builder := newCanalEntryBuilder() - entry, err := builder.fromRowEvent(testCaseDelete, false) - require.Nil(t, err) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := newCanalEntryBuilder(codecConfig) + entry, err := builder.fromRowEvent(event, false) + require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() - require.Equal(t, testCaseDelete.Table.Schema, header.GetSchemaName()) - require.Equal(t, testCaseDelete.Table.Table, header.GetTableName()) + require.Equal(t, event.Table.Schema, header.GetSchemaName()) + require.Equal(t, event.Table.Table, header.GetTableName()) require.Equal(t, canal.EventType_DELETE, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) rc := &canal.RowChange{} err = proto.Unmarshal(store, rc) - require.Nil(t, err) + require.NoError(t, err) require.False(t, rc.GetIsDdl()) rowDatas := rc.GetRowDatas() require.Equal(t, 1, len(rowDatas)) columns := rowDatas[0].BeforeColumns - require.Equal(t, len(testCaseDelete.PreColumns), len(columns)) + require.Equal(t, len(event.PreColumns), len(columns)) for _, col := range columns { require.False(t, col.GetUpdated()) switch col.GetName() { @@ -250,8 +266,8 @@ func testDelete(t *testing.T) { } } -func testDdl(t *testing.T) { - testCaseDdl := &model.DDLEvent{ +func TestDDL(t *testing.T) { + event := &model.DDLEvent{ CommitTs: 417318403368288260, TableInfo: &model.TableInfo{ TableName: model.TableName{ @@ -261,19 +277,19 @@ func testDdl(t *testing.T) { Query: "create table person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))", Type: mm.ActionCreateTable, } - builder := newCanalEntryBuilder() - entry, err := builder.fromDDLEvent(testCaseDdl) - require.Nil(t, err) + builder := newCanalEntryBuilder(nil) + entry, err := builder.fromDDLEvent(event) + require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() - require.Equal(t, testCaseDdl.TableInfo.TableName.Schema, header.GetSchemaName()) - require.Equal(t, testCaseDdl.TableInfo.TableName.Table, header.GetTableName()) + require.Equal(t, event.TableInfo.TableName.Schema, header.GetSchemaName()) + require.Equal(t, event.TableInfo.TableName.Table, header.GetTableName()) require.Equal(t, canal.EventType_CREATE, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) rc := &canal.RowChange{} err = proto.Unmarshal(store, rc) - require.Nil(t, err) + require.NoError(t, err) require.True(t, rc.GetIsDdl()) - require.Equal(t, testCaseDdl.TableInfo.TableName.Schema, rc.GetDdlSchemaName()) + require.Equal(t, event.TableInfo.TableName.Schema, rc.GetDdlSchemaName()) } diff --git a/cdc/sink/codec/canal/canal_json_decoder.go b/cdc/sink/codec/canal/canal_json_decoder.go index 188badd506b..1eb06dfec5a 100644 --- a/cdc/sink/codec/canal/canal_json_decoder.go +++ b/cdc/sink/codec/canal/canal_json_decoder.go @@ -102,7 +102,6 @@ func (b *batchDecoder) HasNext() (model.MessageType, bool, error) { return model.MessageTypeUnknown, false, err } b.msg = msg - return b.msg.messageType(), true, nil } diff --git a/cdc/sink/codec/canal/canal_json_decoder_test.go b/cdc/sink/codec/canal/canal_json_decoder_test.go index a6646a10f36..ebbb53e18e1 100644 --- a/cdc/sink/codec/canal/canal_json_decoder_test.go +++ b/cdc/sink/codec/canal/canal_json_decoder_test.go @@ -24,19 +24,16 @@ import ( ) func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { - t.Parallel() - ctx := context.Background() expectedDecodedValue := collectExpectedDecodedValue(testColumnsTable) for _, encodeEnable := range []bool{false, true} { - encoder := newJSONBatchEncoder(&common.Config{ - EnableTiDBExtension: encodeEnable, - Terminator: config.CRLF, - MaxMessageBytes: config.DefaultMaxMessageBytes, - }) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = encodeEnable + encoder := newJSONBatchEncoder(codecConfig) require.NotNil(t, encoder) - err := encoder.AppendRowChangedEvent(context.Background(), "", testCaseInsert, nil) + insertEvent, _, _ := newLargeEvent4Test(t) + err := encoder.AppendRowChangedEvent(context.Background(), "", insertEvent, nil) require.Nil(t, err) messages := encoder.Build() @@ -52,16 +49,16 @@ func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { require.NoError(t, err) ty, hasNext, err := decoder.HasNext() - require.Nil(t, err) + require.NoError(t, err) require.True(t, hasNext) require.Equal(t, model.MessageTypeRow, ty) consumed, err := decoder.NextRowChangedEvent() require.Nil(t, err) - require.Equal(t, testCaseInsert.Table, consumed.Table) + require.Equal(t, insertEvent.Table, consumed.Table) if encodeEnable && decodeEnable { - require.Equal(t, testCaseInsert.CommitTs, consumed.CommitTs) + require.Equal(t, insertEvent.CommitTs, consumed.CommitTs) } else { require.Equal(t, uint64(0), consumed.CommitTs) } @@ -71,7 +68,7 @@ func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { require.True(t, ok) require.Equal(t, expected, col.Value) - for _, item := range testCaseInsert.Columns { + for _, item := range insertEvent.Columns { if item.Name == col.Name { require.Equal(t, item.Type, col.Type) } @@ -93,17 +90,14 @@ func TestNewCanalJSONBatchDecoder4DDLMessage(t *testing.T) { ctx := context.Background() for _, encodeEnable := range []bool{false, true} { - encoder := &JSONBatchEncoder{ - builder: newCanalEntryBuilder(), - config: &common.Config{ - EnableTiDBExtension: encodeEnable, - LargeMessageHandle: config.NewDefaultLargeMessageHandleConfig(), - }, - } + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = encodeEnable + codecConfig.LargeMessageHandle = config.NewDefaultLargeMessageHandleConfig() + encoder := newJSONBatchEncoder(codecConfig) require.NotNil(t, encoder) result, err := encoder.EncodeDDLEvent(testCaseDDL) - require.Nil(t, err) + require.NoError(t, err) require.NotNil(t, result) for _, decodeEnable := range []bool{false, true} { diff --git a/cdc/sink/codec/canal/canal_json_encoder.go b/cdc/sink/codec/canal/canal_json_encoder.go index a57f6fccd13..6235d0cffec 100644 --- a/cdc/sink/codec/canal/canal_json_encoder.go +++ b/cdc/sink/codec/canal/canal_json_encoder.go @@ -14,6 +14,7 @@ package canal import ( + "bytes" "context" "time" @@ -37,21 +38,10 @@ type JSONBatchEncoder struct { config *common.Config } -// newJSONRowEventEncoder creates a new JSONRowEventEncoder -func newJSONRowEventEncoder(config *common.Config) codec.EventBatchEncoder { - encoder := &JSONBatchEncoder{ - builder: newCanalEntryBuilder(), - messages: make([]*common.Message, 0, 1), - - config: config, - } - return encoder -} - // newJSONBatchEncoder creates a new JSONBatchEncoder func newJSONBatchEncoder(config *common.Config) codec.EventBatchEncoder { encoder := &JSONBatchEncoder{ - builder: newCanalEntryBuilder(), + builder: newCanalEntryBuilder(config), messages: make([]*common.Message, 0, 1), config: config, } @@ -59,7 +49,12 @@ func newJSONBatchEncoder(config *common.Config) codec.EventBatchEncoder { } func fillColumns( - columns []*model.Column, out *jwriter.Writer, onlyHandleKeyColumns bool, builder *canalEntryBuilder, + columns []*model.Column, + out *jwriter.Writer, + onlyHandleKeyColumns bool, + onlyUpdatedColumns bool, + builder *canalEntryBuilder, + newColsMap map[string]*model.Column, ) error { if len(columns) == 0 { out.RawString("null") @@ -73,16 +68,15 @@ func fillColumns( if onlyHandleKeyColumns && !col.Flag.IsHandleKey() { continue } + if onlyUpdatedColumns && shouldIgnoreColumn(col, newColsMap) { + continue + } if isFirst { isFirst = false } else { out.RawByte(',') } - javaType, err := getJavaSQLType(col.Value, col.Type, col.Flag) - if err != nil { - return cerror.WrapError(cerror.ErrCanalEncodeFailed, err) - } - value, err := builder.formatValue(col.Value, javaType) + value, err := builder.formatValue(col.Value, col.Flag.IsBinary()) if err != nil { return cerror.WrapError(cerror.ErrCanalEncodeFailed, err) } @@ -178,7 +172,7 @@ func newJSONMessageForDML( const prefix string = ",\"sqlType\":" out.RawString(prefix) emptyColumn := true - for _, col := range columns { + for idx, col := range columns { if col != nil { if onlyHandleKey && !col.Flag.IsHandleKey() { continue @@ -196,7 +190,12 @@ func newJSONMessageForDML( out.String(col.Name) out.RawByte(':') out.Int32(int32(javaType)) - mysqlTypeMap[col.Name] = getMySQLType(col.Type, col.Flag) + columnInfo, ok := e.TableInfo.GetColumnInfo(e.ColInfos[idx].ID) + if !ok { + return nil, cerror.ErrCanalEncodeFailed.GenWithStack( + "cannot found the column info by the column ID: %d", e.ColInfos[idx].ID) + } + mysqlTypeMap[col.Name] = common.GetMySQLType(columnInfo, config.ContentCompatible) } } if emptyColumn { @@ -230,22 +229,29 @@ func newJSONMessageForDML( if e.IsDelete() { out.RawString(",\"old\":null") out.RawString(",\"data\":") - if err := fillColumns(e.PreColumns, out, onlyHandleKey, builder); err != nil { + if err := fillColumns(e.PreColumns, out, onlyHandleKey, false, builder, nil); err != nil { return nil, err } } else if e.IsInsert() { out.RawString(",\"old\":null") out.RawString(",\"data\":") - if err := fillColumns(e.Columns, out, onlyHandleKey, builder); err != nil { + if err := fillColumns(e.Columns, out, onlyHandleKey, false, builder, nil); err != nil { return nil, err } } else if e.IsUpdate() { + var newColsMap map[string]*model.Column + if config.ContentCompatible { + newColsMap = make(map[string]*model.Column, len(e.Columns)) + for _, col := range e.Columns { + newColsMap[col.Name] = col + } + } out.RawString(",\"old\":") - if err := fillColumns(e.PreColumns, out, onlyHandleKey, builder); err != nil { + if err := fillColumns(e.PreColumns, out, onlyHandleKey, config.ContentCompatible, builder, newColsMap); err != nil { return nil, err } out.RawString(",\"data\":") - if err := fillColumns(e.Columns, out, onlyHandleKey, builder); err != nil { + if err := fillColumns(e.Columns, out, onlyHandleKey, false, builder, nil); err != nil { return nil, err } } else { @@ -423,3 +429,35 @@ func NewJSONBatchEncoderBuilder(config *common.Config) codec.EncoderBuilder { func (b *jsonBatchEncoderBuilder) Build() codec.EventBatchEncoder { return newJSONBatchEncoder(b.config) } + +func shouldIgnoreColumn(col *model.Column, + newColumnMap map[string]*model.Column, +) bool { + newCol, ok := newColumnMap[col.Name] + if ok && newCol != nil { + // sql type is not equal + if newCol.Type != col.Type { + return false + } + // value equal + if isColumnValueEqual(newCol.Value, col.Value) { + return true + } + } + return false +} + +func isColumnValueEqual(preValue, updatedValue interface{}) bool { + if preValue == nil || updatedValue == nil { + return preValue == updatedValue + } + + preValueBytes, ok1 := preValue.([]byte) + updatedValueBytes, ok2 := updatedValue.([]byte) + if ok1 && ok2 { + return bytes.Equal(preValueBytes, updatedValueBytes) + } + // mounter use the same table info to parse the value, + // the value type should be the same + return preValue == updatedValue +} diff --git a/cdc/sink/codec/canal/canal_json_encoder_test.go b/cdc/sink/codec/canal/canal_json_encoder_test.go index 91a117e3e76..5de3c9bd37f 100644 --- a/cdc/sink/codec/canal/canal_json_encoder_test.go +++ b/cdc/sink/codec/canal/canal_json_encoder_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec/common" "github.com/pingcap/tiflow/pkg/config" @@ -30,24 +31,21 @@ func TestBuildJSONBatchEncoder(t *testing.T) { t.Parallel() cfg := common.NewConfig(config.ProtocolCanalJSON) - builder := &jsonBatchEncoderBuilder{config: cfg} + builder := NewJSONBatchEncoderBuilder(cfg) encoder, ok := builder.Build().(*JSONBatchEncoder) require.True(t, ok) require.NotNil(t, encoder.config) } func TestNewCanalJSONMessage4DML(t *testing.T) { - t.Parallel() - e := newJSONBatchEncoder(&common.Config{ - EnableTiDBExtension: false, - Terminator: "", - }) - require.NotNil(t, e) - - encoder, ok := e.(*JSONBatchEncoder) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true + builder := NewJSONBatchEncoderBuilder(codecConfig) + encoder, ok := builder.Build().(*JSONBatchEncoder) require.True(t, ok) - data, err := newJSONMessageForDML(testCaseInsert, encoder.config, encoder.builder, false) + insertEvent, updateEvent, deleteEvent := newLargeEvent4Test(t) + data, err := newJSONMessageForDML(insertEvent, encoder.config, encoder.builder, false) require.Nil(t, err) var msg canalJSONMessageInterface = &JSONMessage{} err = json.Unmarshal(data, msg) @@ -57,12 +55,12 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { require.NotNil(t, jsonMsg.Data) require.Nil(t, jsonMsg.Old) require.Equal(t, "INSERT", jsonMsg.EventType) - require.Equal(t, convertToCanalTs(testCaseInsert.CommitTs), jsonMsg.ExecutionTime) - require.Equal(t, "cdc", jsonMsg.Schema) - require.Equal(t, "person", jsonMsg.Table) + require.Equal(t, convertToCanalTs(insertEvent.CommitTs), jsonMsg.ExecutionTime) + require.Equal(t, "test", jsonMsg.Schema) + require.Equal(t, "t", jsonMsg.Table) require.False(t, jsonMsg.IsDDL) - for _, col := range testCaseInsert.Columns { + for _, col := range insertEvent.Columns { require.Contains(t, jsonMsg.Data[0], col.Name) require.Contains(t, jsonMsg.SQLType, col.Name) require.Contains(t, jsonMsg.MySQLType, col.Name) @@ -96,7 +94,7 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { require.Equal(t, item.expectedEncodedValue, obtainedValue) } - data, err = newJSONMessageForDML(testCaseUpdate, encoder.config, encoder.builder, false) + data, err = newJSONMessageForDML(updateEvent, encoder.config, encoder.builder, false) require.NoError(t, err) jsonMsg = &JSONMessage{} err = json.Unmarshal(data, jsonMsg) @@ -105,16 +103,16 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { require.NotNil(t, jsonMsg.Old) require.Equal(t, "UPDATE", jsonMsg.EventType) - for _, col := range testCaseUpdate.Columns { + for _, col := range updateEvent.Columns { require.Contains(t, jsonMsg.Data[0], col.Name) require.Contains(t, jsonMsg.SQLType, col.Name) require.Contains(t, jsonMsg.MySQLType, col.Name) } - for _, col := range testCaseUpdate.PreColumns { + for _, col := range updateEvent.PreColumns { require.Contains(t, jsonMsg.Old[0], col.Name) } - data, err = newJSONMessageForDML(testCaseDelete, encoder.config, encoder.builder, false) + data, err = newJSONMessageForDML(deleteEvent, encoder.config, encoder.builder, false) require.Nil(t, err) jsonMsg = &JSONMessage{} err = json.Unmarshal(data, jsonMsg) @@ -123,11 +121,11 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { require.Nil(t, jsonMsg.Old) require.Equal(t, "DELETE", jsonMsg.EventType) - for _, col := range testCaseDelete.PreColumns { + for _, col := range deleteEvent.PreColumns { require.Contains(t, jsonMsg.Data[0], col.Name) } - data, err = newJSONMessageForDML(testCaseDelete, encoder.config, encoder.builder, false) + data, err = newJSONMessageForDML(deleteEvent, encoder.config, encoder.builder, false) require.NoError(t, err) jsonMsg = &JSONMessage{} err = json.Unmarshal(data, jsonMsg) @@ -135,7 +133,7 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { require.NotNil(t, jsonMsg.Data) require.Nil(t, jsonMsg.Old) - for _, col := range testCaseDelete.PreColumns { + for _, col := range deleteEvent.PreColumns { require.Contains(t, jsonMsg.Data[0], col.Name) require.Contains(t, jsonMsg.SQLType, col.Name) require.Contains(t, jsonMsg.MySQLType, col.Name) @@ -143,7 +141,7 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { encoder, ok = newJSONBatchEncoder(&common.Config{DeleteOnlyHandleKeyColumns: true}).(*JSONBatchEncoder) require.True(t, ok) - data, err = newJSONMessageForDML(testCaseDelete, encoder.config, encoder.builder, false) + data, err = newJSONMessageForDML(deleteEvent, encoder.config, encoder.builder, false) require.NoError(t, err) jsonMsg = &JSONMessage{} err = json.Unmarshal(data, jsonMsg) @@ -151,7 +149,7 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { require.NotNil(t, jsonMsg.Data) require.Nil(t, jsonMsg.Old) - for _, col := range testCaseDelete.PreColumns { + for _, col := range deleteEvent.PreColumns { if col.Flag.IsHandleKey() { require.Contains(t, jsonMsg.Data[0], col.Name) require.Contains(t, jsonMsg.SQLType, col.Name) @@ -163,15 +161,9 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { } } - e = newJSONRowEventEncoder(&common.Config{ - EnableTiDBExtension: true, - Terminator: "", - }) - require.NotNil(t, e) - - encoder, ok = e.(*JSONBatchEncoder) + encoder, ok = NewJSONBatchEncoderBuilder(codecConfig).Build().(*JSONBatchEncoder) require.True(t, ok) - data, err = newJSONMessageForDML(testCaseUpdate, encoder.config, encoder.builder, false) + data, err = newJSONMessageForDML(updateEvent, encoder.config, encoder.builder, false) require.Nil(t, err) withExtension := &canalJSONMessageWithTiDBExtension{} @@ -179,19 +171,35 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { require.Nil(t, err) require.NotNil(t, withExtension.Extensions) - require.Equal(t, testCaseUpdate.CommitTs, withExtension.Extensions.CommitTs) + require.Equal(t, updateEvent.CommitTs, withExtension.Extensions.CommitTs) + + codecConfig = common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true + codecConfig.ContentCompatible = true + encoder, ok = NewJSONBatchEncoderBuilder(codecConfig).Build().(*JSONBatchEncoder) + require.True(t, ok) + + data, err = newJSONMessageForDML(updateEvent, encoder.config, encoder.builder, false) + require.NoError(t, err) + + withExtension = &canalJSONMessageWithTiDBExtension{} + err = json.Unmarshal(data, withExtension) + require.Nil(t, err) + require.Equal(t, 0, len(withExtension.JSONMessage.Old[0])) + + require.NotNil(t, withExtension.Extensions) + require.Equal(t, updateEvent.CommitTs, withExtension.Extensions.CommitTs) } func TestNewCanalJSONMessageHandleKeyOnly4LargeMessage(t *testing.T) { - t.Parallel() - codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = true codecConfig.LargeMessageHandle.LargeMessageHandleOption = config.LargeMessageHandleOptionHandleKeyOnly codecConfig.MaxMessageBytes = 500 - encoder := newJSONRowEventEncoder(codecConfig) + encoder := newJSONBatchEncoder(codecConfig) - err := encoder.AppendRowChangedEvent(context.Background(), "", testCaseInsert, func() {}) + insertEvent, _, _ := newLargeEvent4Test(t) + err := encoder.AppendRowChangedEvent(context.Background(), "", insertEvent, func() {}) require.NoError(t, err) message := encoder.Build()[0] @@ -201,7 +209,7 @@ func TestNewCanalJSONMessageHandleKeyOnly4LargeMessage(t *testing.T) { require.NoError(t, err) require.True(t, decoded.Extensions.OnlyHandleKey) - for _, col := range testCaseInsert.Columns { + for _, col := range insertEvent.Columns { if col.Flag.IsHandleKey() { require.Contains(t, decoded.Data[0], col.Name) require.Contains(t, decoded.SQLType, col.Name) @@ -217,7 +225,7 @@ func TestNewCanalJSONMessageHandleKeyOnly4LargeMessage(t *testing.T) { func TestNewCanalJSONMessageFromDDL(t *testing.T) { t.Parallel() - encoder, ok := newJSONRowEventEncoder(&common.Config{}).(*JSONBatchEncoder) + encoder, ok := newJSONBatchEncoder(&common.Config{}).(*JSONBatchEncoder) require.True(t, ok) message := encoder.newJSONMessageForDDL(testCaseDDL) @@ -232,7 +240,7 @@ func TestNewCanalJSONMessageFromDDL(t *testing.T) { require.Equal(t, testCaseDDL.Query, msg.Query) require.Equal(t, "CREATE", msg.EventType) - encoder, ok = newJSONRowEventEncoder(&common.Config{ + encoder, ok = newJSONBatchEncoder(&common.Config{ EnableTiDBExtension: true, }).(*JSONBatchEncoder) require.True(t, ok) @@ -248,15 +256,12 @@ func TestNewCanalJSONMessageFromDDL(t *testing.T) { } func TestBatching(t *testing.T) { - t.Parallel() - encoder := newJSONBatchEncoder(&common.Config{ - EnableTiDBExtension: false, - Terminator: "", - MaxMessageBytes: config.DefaultMaxMessageBytes, - }) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + encoder := newJSONBatchEncoder(codecConfig) require.NotNil(t, encoder) - updateCase := *testCaseUpdate + _, updateEvent, _ := newLargeEvent4Test(t) + updateCase := *updateEvent for i := 1; i <= 1000; i++ { ts := uint64(i) updateCase.CommitTs = ts @@ -375,10 +380,11 @@ func TestCheckpointEventValueMarshal(t *testing.T) { func TestDDLEventWithExtensionValueMarshal(t *testing.T) { t.Parallel() - encoder := &JSONBatchEncoder{ - builder: newCanalEntryBuilder(), - config: &common.Config{EnableTiDBExtension: true}, - } + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true + + builder := NewJSONBatchEncoderBuilder(codecConfig) + encoder := builder.Build().(*JSONBatchEncoder) require.NotNil(t, encoder) message := encoder.newJSONMessageForDDL(testCaseDDL) @@ -415,6 +421,15 @@ func TestDDLEventWithExtensionValueMarshal(t *testing.T) { } func TestCanalJSONAppendRowChangedEventWithCallback(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + encoder := newJSONBatchEncoder(&common.Config{ EnableTiDBExtension: true, Terminator: "", @@ -425,13 +440,15 @@ func TestCanalJSONAppendRowChangedEventWithCallback(t *testing.T) { count := 0 row := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa"), }}, + ColInfos: colInfos, } tests := []struct { @@ -496,15 +513,26 @@ func TestCanalJSONAppendRowChangedEventWithCallback(t *testing.T) { } func TestMaxMessageBytes(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + // the size of `testEvent` after being encoded by canal-json is 200 testEvent := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa"), }}, + ColInfos: colInfos, } ctx := context.Background() @@ -524,3 +552,49 @@ func TestMaxMessageBytes(t *testing.T) { err = encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) require.NotNil(t, err) } + +func TestCanalJSONContentCompatibleE2E(t *testing.T) { + ctx := context.Background() + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true + codecConfig.ContentCompatible = true + + builder := NewJSONBatchEncoderBuilder(codecConfig) + encoder := builder.Build() + + insertEvent, _, _ := newLargeEvent4Test(t) + err := encoder.AppendRowChangedEvent(ctx, "", insertEvent, func() {}) + require.NoError(t, err) + + message := encoder.Build()[0] + + decoder, err := NewBatchDecoder(ctx, codecConfig, nil) + require.NoError(t, err) + + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + + messageType, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, messageType, model.MessageTypeRow) + + decodedEvent, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + require.Equal(t, decodedEvent.CommitTs, insertEvent.CommitTs) + require.Equal(t, decodedEvent.Table.Schema, insertEvent.Table.Schema) + require.Equal(t, decodedEvent.Table.Table, insertEvent.Table.Table) + + obtainedColumns := make(map[string]*model.Column, len(decodedEvent.Columns)) + for _, column := range decodedEvent.Columns { + obtainedColumns[column.Name] = column + } + + expectedValue := collectExpectedDecodedValue(testColumnsTable) + for _, actual := range insertEvent.Columns { + obtained, ok := obtainedColumns[actual.Name] + require.True(t, ok) + require.Equal(t, actual.Type, obtained.Type) + require.Equal(t, expectedValue[actual.Name], obtained.Value) + } +} diff --git a/cdc/sink/codec/canal/canal_json_message.go b/cdc/sink/codec/canal/canal_json_message.go index 61a326a96ac..fb2f13a686f 100644 --- a/cdc/sink/codec/canal/canal_json_message.go +++ b/cdc/sink/codec/canal/canal_json_message.go @@ -15,14 +15,18 @@ package canal import ( "sort" + "strconv" "strings" + "github.com/pingcap/log" timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sink/codec/internal" cerrors "github.com/pingcap/tiflow/pkg/errors" canal "github.com/pingcap/tiflow/proto/canal" + "go.uber.org/zap" + "golang.org/x/text/encoding/charmap" ) const tidbWaterMarkType = "TIDB_WATERMARK" @@ -196,11 +200,7 @@ func canalJSONColumnMap2RowChangeColumns(cols map[string]interface{}, mysqlType return nil, cerrors.ErrCanalDecodeFailed.GenWithStack( "mysql type does not found, column: %+v, mysqlType: %+v", name, mysqlType) } - mysqlTypeStr = trimUnsignedFromMySQLType(mysqlTypeStr) - isBinary := isBinaryMySQLType(mysqlTypeStr) - mysqlType := types.StrToType(mysqlTypeStr) - col := internal.NewColumn(value, mysqlType). - ToCanalJSONFormatColumn(name, isBinary) + col := canalJSONFormatColumn(value, name, mysqlTypeStr) result = append(result, col) } if len(result) == 0 { @@ -212,6 +212,60 @@ func canalJSONColumnMap2RowChangeColumns(cols map[string]interface{}, mysqlType return result, nil } +func extractBasicMySQLType(mysqlType string) string { + for i := 0; i < len(mysqlType); i++ { + if mysqlType[i] == '(' || mysqlType[i] == ' ' { + return mysqlType[:i] + } + } + return mysqlType +} + +func canalJSONFormatColumn(value interface{}, name string, mysqlTypeStr string) *model.Column { + mysqlTypeStr = extractBasicMySQLType(mysqlTypeStr) + mysqlType := types.StrToType(mysqlTypeStr) + + result := &model.Column{ + Type: mysqlType, + Name: name, + Value: value, + } + if result.Value == nil { + return result + } + + data, ok := value.(string) + if !ok { + log.Panic("canal-json encoded message should have type in `string`") + } + + if mysqlType == mysql.TypeBit || mysqlType == mysql.TypeSet { + val, err := strconv.ParseUint(data, 10, 64) + if err != nil { + log.Panic("invalid column value for bit", zap.Any("col", result), zap.Error(err)) + } + result.Value = val + return result + } + + var err error + if isBinaryMySQLType(mysqlTypeStr) { + // when encoding the `JavaSQLTypeBLOB`, use `ISO8859_1` decoder, now reverse it back. + encoder := charmap.ISO8859_1.NewEncoder() + value, err = encoder.String(data) + if err != nil { + log.Panic("invalid column value, please report a bug", zap.Any("col", result), zap.Error(err)) + } + } + + result.Value = value + return result +} + +func isBinaryMySQLType(mysqlType string) bool { + return strings.Contains(mysqlType, "blob") || strings.Contains(mysqlType, "binary") +} + func canalJSONMessage2DDLEvent(msg canalJSONMessageInterface) *model.DDLEvent { result := new(model.DDLEvent) // we lost the startTs from kafka message @@ -245,7 +299,3 @@ func getDDLActionType(query string) timodel.ActionType { return timodel.ActionNone } - -func isBinaryMySQLType(mysqlType string) bool { - return strings.Contains(mysqlType, "blob") || strings.Contains(mysqlType, "binary") -} diff --git a/cdc/sink/codec/canal/canal_test_util.go b/cdc/sink/codec/canal/canal_test_util.go index 578653e0685..c7d3d82ba03 100644 --- a/cdc/sink/codec/canal/canal_test_util.go +++ b/cdc/sink/codec/canal/canal_test_util.go @@ -14,16 +14,16 @@ package canal import ( + "testing" + mm "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sink/codec/internal" ) type testColumnTuple struct { - column *model.Column - expectedMySQLType string - expectedJavaSQLType internal.JavaSQLType + column *model.Column // expectedEncodedValue is expected by encoding expectedEncodedValue string @@ -35,185 +35,185 @@ type testColumnTuple struct { var ( testColumnsTable = []*testColumnTuple{ { - &model.Column{Name: "tinyint", Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, Type: mysql.TypeTiny, Value: int64(127)}, - "tinyint", internal.JavaSQLTypeTINYINT, "127", "127", + &model.Column{Name: "t", Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, Type: mysql.TypeTiny, Value: int64(127)}, + "127", "127", }, { &model.Column{ - Name: "tinyint unsigned", Type: mysql.TypeTiny, Value: uint64(127), + Name: "tu1", Type: mysql.TypeTiny, Value: uint64(127), Flag: model.UnsignedFlag, }, - "tinyint unsigned", internal.JavaSQLTypeTINYINT, "127", "127", + "127", "127", }, { &model.Column{ - Name: "tinyint unsigned 2", Type: mysql.TypeTiny, Value: uint64(128), + Name: "tu2", Type: mysql.TypeTiny, Value: uint64(128), Flag: model.UnsignedFlag, }, - "tinyint unsigned", internal.JavaSQLTypeSMALLINT, "128", "128", + "128", "128", }, { &model.Column{ - Name: "tinyint unsigned 3", Type: mysql.TypeTiny, Value: "0", + Name: "tu3", Type: mysql.TypeTiny, Value: "0", Flag: model.UnsignedFlag, }, - "tinyint unsigned", internal.JavaSQLTypeTINYINT, "0", "0", + "0", "0", }, { &model.Column{ - Name: "tinyint unsigned 4", Type: mysql.TypeTiny, Value: nil, + Name: "tu4", Type: mysql.TypeTiny, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "tinyint unsigned", internal.JavaSQLTypeTINYINT, "", nil, + "", nil, }, { - &model.Column{Name: "smallint", Type: mysql.TypeShort, Value: int64(32767)}, - "smallint", internal.JavaSQLTypeSMALLINT, "32767", "32767", + &model.Column{Name: "s", Type: mysql.TypeShort, Value: int64(32767)}, + "32767", "32767", }, { &model.Column{ - Name: "smallint unsigned", Type: mysql.TypeShort, Value: uint64(32767), + Name: "su1", Type: mysql.TypeShort, Value: uint64(32767), Flag: model.UnsignedFlag, }, - "smallint unsigned", internal.JavaSQLTypeSMALLINT, "32767", "32767", + "32767", "32767", }, { &model.Column{ - Name: "smallint unsigned 2", Type: mysql.TypeShort, Value: uint64(32768), + Name: "su2", Type: mysql.TypeShort, Value: uint64(32768), Flag: model.UnsignedFlag, }, - "smallint unsigned", internal.JavaSQLTypeINTEGER, "32768", "32768", + "32768", "32768", }, { &model.Column{ - Name: "smallint unsigned 3", Type: mysql.TypeShort, Value: "0", + Name: "su3", Type: mysql.TypeShort, Value: "0", Flag: model.UnsignedFlag, }, - "smallint unsigned", internal.JavaSQLTypeSMALLINT, "0", "0", + "0", "0", }, { &model.Column{ - Name: "smallint unsigned 4", Type: mysql.TypeShort, Value: nil, + Name: "su4", Type: mysql.TypeShort, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "smallint unsigned", internal.JavaSQLTypeSMALLINT, "", nil, + "", nil, }, { - &model.Column{Name: "mediumint", Type: mysql.TypeInt24, Value: int64(8388607)}, - "mediumint", internal.JavaSQLTypeINTEGER, "8388607", "8388607", + &model.Column{Name: "m", Type: mysql.TypeInt24, Value: int64(8388607)}, + "8388607", "8388607", }, { &model.Column{ - Name: "mediumint unsigned", Type: mysql.TypeInt24, Value: uint64(8388607), + Name: "mu1", Type: mysql.TypeInt24, Value: uint64(8388607), Flag: model.UnsignedFlag, }, - "mediumint unsigned", internal.JavaSQLTypeINTEGER, "8388607", "8388607", + "8388607", "8388607", }, { &model.Column{ - Name: "mediumint unsigned 2", Type: mysql.TypeInt24, Value: uint64(8388608), + Name: "mu2", Type: mysql.TypeInt24, Value: uint64(8388608), Flag: model.UnsignedFlag, }, - "mediumint unsigned", internal.JavaSQLTypeINTEGER, "8388608", "8388608", + "8388608", "8388608", }, { &model.Column{ - Name: "mediumint unsigned 3", Type: mysql.TypeInt24, Value: "0", + Name: "mu3", Type: mysql.TypeInt24, Value: "0", Flag: model.UnsignedFlag, }, - "mediumint unsigned", internal.JavaSQLTypeINTEGER, "0", "0", + "0", "0", }, { &model.Column{ - Name: "mediumint unsigned 4", Type: mysql.TypeInt24, Value: nil, + Name: "mu4", Type: mysql.TypeInt24, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "mediumint unsigned", internal.JavaSQLTypeINTEGER, "", nil, + "", nil, }, { - &model.Column{Name: "int", Type: mysql.TypeLong, Value: int64(2147483647)}, - "int", internal.JavaSQLTypeINTEGER, "2147483647", "2147483647", + &model.Column{Name: "i", Type: mysql.TypeLong, Value: int64(2147483647)}, + "2147483647", "2147483647", }, { &model.Column{ - Name: "int unsigned", Type: mysql.TypeLong, Value: uint64(2147483647), + Name: "iu1", Type: mysql.TypeLong, Value: uint64(2147483647), Flag: model.UnsignedFlag, }, - "int unsigned", internal.JavaSQLTypeINTEGER, "2147483647", "2147483647", + "2147483647", "2147483647", }, { &model.Column{ - Name: "int unsigned 2", Type: mysql.TypeLong, Value: uint64(2147483648), + Name: "iu2", Type: mysql.TypeLong, Value: uint64(2147483648), Flag: model.UnsignedFlag, }, - "int unsigned", internal.JavaSQLTypeBIGINT, "2147483648", "2147483648", + "2147483648", "2147483648", }, { &model.Column{ - Name: "int unsigned 3", Type: mysql.TypeLong, Value: "0", + Name: "iu3", Type: mysql.TypeLong, Value: "0", Flag: model.UnsignedFlag, }, - "int unsigned", internal.JavaSQLTypeINTEGER, "0", "0", + "0", "0", }, { &model.Column{ - Name: "int unsigned 4", Type: mysql.TypeLong, Value: nil, + Name: "iu4", Type: mysql.TypeLong, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "int unsigned", internal.JavaSQLTypeINTEGER, "", nil, + "", nil, }, { - &model.Column{Name: "bigint", Type: mysql.TypeLonglong, Value: int64(9223372036854775807)}, - "bigint", internal.JavaSQLTypeBIGINT, "9223372036854775807", "9223372036854775807", + &model.Column{Name: "bi", Type: mysql.TypeLonglong, Value: int64(9223372036854775807)}, + "9223372036854775807", "9223372036854775807", }, { &model.Column{ - Name: "bigint unsigned", Type: mysql.TypeLonglong, Value: uint64(9223372036854775807), + Name: "biu1", Type: mysql.TypeLonglong, Value: uint64(9223372036854775807), Flag: model.UnsignedFlag, }, - "bigint unsigned", internal.JavaSQLTypeBIGINT, "9223372036854775807", "9223372036854775807", + "9223372036854775807", "9223372036854775807", }, { &model.Column{ - Name: "bigint unsigned 2", Type: mysql.TypeLonglong, Value: uint64(9223372036854775808), + Name: "biu2", Type: mysql.TypeLonglong, Value: uint64(9223372036854775808), Flag: model.UnsignedFlag, }, - "bigint unsigned", internal.JavaSQLTypeDECIMAL, "9223372036854775808", "9223372036854775808", + "9223372036854775808", "9223372036854775808", }, { &model.Column{ - Name: "bigint unsigned 3", Type: mysql.TypeLonglong, Value: "0", + Name: "biu3", Type: mysql.TypeLonglong, Value: "0", Flag: model.UnsignedFlag, }, - "bigint unsigned", internal.JavaSQLTypeBIGINT, "0", "0", + "0", "0", }, { &model.Column{ - Name: "bigint unsigned 4", Type: mysql.TypeLonglong, Value: nil, + Name: "biu4", Type: mysql.TypeLonglong, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "bigint unsigned", internal.JavaSQLTypeBIGINT, "", nil, + "", nil, }, { - &model.Column{Name: "float", Type: mysql.TypeFloat, Value: 3.14}, - "float", internal.JavaSQLTypeREAL, "3.14", "3.14", + &model.Column{Name: "floatT", Type: mysql.TypeFloat, Value: 3.14}, + "3.14", "3.14", }, { - &model.Column{Name: "double", Type: mysql.TypeDouble, Value: 2.71}, - "double", internal.JavaSQLTypeDOUBLE, "2.71", "2.71", + &model.Column{Name: "doubleT", Type: mysql.TypeDouble, Value: 2.71}, + "2.71", "2.71", }, { - &model.Column{Name: "decimal", Type: mysql.TypeNewDecimal, Value: "2333"}, - "decimal", internal.JavaSQLTypeDECIMAL, "2333", "2333", + &model.Column{Name: "decimalT", Type: mysql.TypeNewDecimal, Value: "2333"}, + "2333", "2333", }, { @@ -221,264 +221,264 @@ var ( Name: "float unsigned", Type: mysql.TypeFloat, Value: 3.14, Flag: model.UnsignedFlag, }, - "float unsigned", internal.JavaSQLTypeREAL, "3.14", "3.14", + "3.14", "3.14", }, { &model.Column{ Name: "double unsigned", Type: mysql.TypeDouble, Value: 2.71, Flag: model.UnsignedFlag, }, - "double unsigned", internal.JavaSQLTypeDOUBLE, "2.71", "2.71", + "2.71", "2.71", }, { &model.Column{ Name: "decimal unsigned", Type: mysql.TypeNewDecimal, Value: "2333", Flag: model.UnsignedFlag, }, - "decimal unsigned", internal.JavaSQLTypeDECIMAL, "2333", "2333", + "2333", "2333", }, // for column value type in `[]uint8` and have `BinaryFlag`, expectedEncodedValue is dummy. { - &model.Column{Name: "varchar", Type: mysql.TypeVarchar, Value: []uint8("测试Varchar")}, - "varchar", internal.JavaSQLTypeVARCHAR, "测试Varchar", "测试Varchar", + &model.Column{Name: "varcharT", Type: mysql.TypeVarchar, Value: []uint8("测试Varchar")}, + "测试Varchar", "测试Varchar", }, { - &model.Column{Name: "char", Type: mysql.TypeString, Value: []uint8("测试String")}, - "char", internal.JavaSQLTypeCHAR, "测试String", "测试String", + &model.Column{Name: "charT", Type: mysql.TypeString, Value: []uint8("测试String")}, + "测试String", "测试String", }, { &model.Column{ - Name: "binary", Type: mysql.TypeString, Value: []uint8("测试Binary"), + Name: "binaryT", Type: mysql.TypeString, Value: []uint8("测试Binary"), Flag: model.BinaryFlag, }, - "binary", internal.JavaSQLTypeBLOB, "测试Binary", "测试Binary", + "测试Binary", "测试Binary", }, { &model.Column{ - Name: "varbinary", Type: mysql.TypeVarchar, Value: []uint8("测试varbinary"), + Name: "varbinaryT", Type: mysql.TypeVarchar, Value: []uint8("测试varbinary"), Flag: model.BinaryFlag, }, - "varbinary", internal.JavaSQLTypeBLOB, "测试varbinary", "测试varbinary", + "测试varbinary", "测试varbinary", }, { - &model.Column{Name: "tinytext", Type: mysql.TypeTinyBlob, Value: []uint8("测试Tinytext")}, - "tinytext", internal.JavaSQLTypeCLOB, "测试Tinytext", "测试Tinytext", + &model.Column{Name: "tinytextT", Type: mysql.TypeTinyBlob, Value: []uint8("测试Tinytext")}, + "测试Tinytext", "测试Tinytext", }, + { - &model.Column{Name: "text", Type: mysql.TypeBlob, Value: []uint8("测试text")}, - "text", internal.JavaSQLTypeCLOB, "测试text", "测试text", + &model.Column{Name: "textT", Type: mysql.TypeBlob, Value: []uint8("测试text")}, + "测试text", "测试text", }, { - &model.Column{ - Name: "mediumtext", Type: mysql.TypeMediumBlob, - Value: []uint8("测试mediumtext"), - }, - "mediumtext", internal.JavaSQLTypeCLOB, "测试mediumtext", "测试mediumtext", + &model.Column{Name: "mediumtextT", Type: mysql.TypeMediumBlob, Value: []uint8("测试mediumtext")}, + "测试mediumtext", "测试mediumtext", }, { - &model.Column{Name: "longtext", Type: mysql.TypeLongBlob, Value: []uint8("测试longtext")}, - "longtext", internal.JavaSQLTypeCLOB, "测试longtext", "测试longtext", + &model.Column{Name: "longtextT", Type: mysql.TypeLongBlob, Value: []uint8("测试longtext")}, + "测试longtext", "测试longtext", }, { &model.Column{ - Name: "tinyblob", Type: mysql.TypeTinyBlob, Value: []uint8("测试tinyblob"), + Name: "tinyblobT", Type: mysql.TypeTinyBlob, Value: []uint8("测试tinyblob"), Flag: model.BinaryFlag, }, - "tinyblob", internal.JavaSQLTypeBLOB, "测试tinyblob", "测试tinyblob", + "测试tinyblob", "测试tinyblob", }, { &model.Column{ - Name: "blob", Type: mysql.TypeBlob, Value: []uint8("测试blob"), + Name: "blobT", Type: mysql.TypeBlob, Value: []uint8("测试blob"), Flag: model.BinaryFlag, }, - "blob", internal.JavaSQLTypeBLOB, "测试blob", "测试blob", + "测试blob", "测试blob", }, { &model.Column{ - Name: "mediumblob", Type: mysql.TypeMediumBlob, Value: []uint8("测试mediumblob"), + Name: "mediumblobT", Type: mysql.TypeMediumBlob, Value: []uint8("测试mediumblob"), Flag: model.BinaryFlag, }, - "mediumblob", internal.JavaSQLTypeBLOB, "测试mediumblob", "测试mediumblob", + "测试mediumblob", "测试mediumblob", }, { &model.Column{ - Name: "longblob", Type: mysql.TypeLongBlob, Value: []uint8("测试longblob"), + Name: "longblobT", Type: mysql.TypeLongBlob, Value: []uint8("测试longblob"), Flag: model.BinaryFlag, }, - "longblob", internal.JavaSQLTypeBLOB, "测试longblob", "测试longblob", + "测试longblob", "测试longblob", }, { - &model.Column{Name: "date", Type: mysql.TypeDate, Value: "2020-02-20"}, - "date", internal.JavaSQLTypeDATE, "2020-02-20", "2020-02-20", + &model.Column{Name: "dateT", Type: mysql.TypeDate, Value: "2020-02-20"}, + "2020-02-20", "2020-02-20", }, { - &model.Column{Name: "datetime", Type: mysql.TypeDatetime, Value: "2020-02-20 02:20:20"}, - "datetime", internal.JavaSQLTypeTIMESTAMP, "2020-02-20 02:20:20", "2020-02-20 02:20:20", + &model.Column{Name: "datetimeT", Type: mysql.TypeDatetime, Value: "2020-02-20 02:20:20"}, + "2020-02-20 02:20:20", "2020-02-20 02:20:20", }, { - &model.Column{Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2020-02-20 10:20:20"}, - "timestamp", internal.JavaSQLTypeTIMESTAMP, "2020-02-20 10:20:20", "2020-02-20 10:20:20", + &model.Column{Name: "timestampT", Type: mysql.TypeTimestamp, Value: "2020-02-20 10:20:20"}, + "2020-02-20 10:20:20", "2020-02-20 10:20:20", }, { - &model.Column{Name: "time", Type: mysql.TypeDuration, Value: "02:20:20"}, - "time", internal.JavaSQLTypeTIME, "02:20:20", "02:20:20", + &model.Column{Name: "timeT", Type: mysql.TypeDuration, Value: "02:20:20"}, + "02:20:20", "02:20:20", }, { - &model.Column{Name: "year", Type: mysql.TypeYear, Value: "2020", Flag: model.UnsignedFlag}, - "year", internal.JavaSQLTypeVARCHAR, "2020", "2020", + &model.Column{Name: "yearT", Type: mysql.TypeYear, Value: "2020", Flag: model.UnsignedFlag}, + "2020", "2020", }, { - &model.Column{Name: "enum", Type: mysql.TypeEnum, Value: uint64(1)}, - "enum", internal.JavaSQLTypeINTEGER, "1", "1", + &model.Column{Name: "enumT", Type: mysql.TypeEnum, Value: uint64(1)}, + "1", "1", }, { - &model.Column{Name: "set", Type: mysql.TypeSet, Value: uint64(3)}, - "set", internal.JavaSQLTypeBIT, "3", uint64(3), + &model.Column{Name: "setT", Type: mysql.TypeSet, Value: uint64(2)}, + "2", uint64(2), }, { &model.Column{ - Name: "bit", Type: mysql.TypeBit, Value: uint64(65), + Name: "bitT", Type: mysql.TypeBit, Value: uint64(65), Flag: model.UnsignedFlag | model.BinaryFlag, }, - "bit", internal.JavaSQLTypeBIT, "65", uint64(65), + "65", uint64(65), }, { &model.Column{ - Name: "json", Type: mysql.TypeJSON, Value: "{\"key1\": \"value1\"}", + Name: "jsonT", Type: mysql.TypeJSON, Value: "{\"key1\": \"value1\"}", Flag: model.BinaryFlag, }, - "json", internal.JavaSQLTypeVARCHAR, "{\"key1\": \"value1\"}", "{\"key1\": \"value1\"}", + "{\"key1\": \"value1\"}", "{\"key1\": \"value1\"}", }, } - defaultCanalBatchTester = &struct { - rowCases [][]*model.RowChangedEvent - ddlCases [][]*model.DDLEvent - }{ - rowCases: [][]*model.RowChangedEvent{ - {{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{ - Name: "col1", - Type: mysql.TypeVarchar, - Value: []byte("aa"), - }}, - }}, - { - { - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{ - Name: "col1", - Type: mysql.TypeVarchar, - Value: []byte("aa"), - }}, - }, - { - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, - }, - }, - }, - ddlCases: [][]*model.DDLEvent{ - {{ - CommitTs: 1, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table a", - Type: 1, - }}, - { - { - CommitTs: 2, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table b", - Type: 3, - }, - { - CommitTs: 3, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table c", - Type: 3, - }, + testCaseDDL = &model.DDLEvent{ + CommitTs: 417318403368288260, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "cdc", Table: "person", }, }, + Query: "create table person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))", + Type: mm.ActionCreateTable, } +) + +func collectAllColumns(groups []*testColumnTuple) []*model.Column { + columns := make([]*model.Column, 0, len(groups)) + for _, item := range groups { + columns = append(columns, item.column) + } + return columns +} + +func collectExpectedDecodedValue(columns []*testColumnTuple) map[string]interface{} { + result := make(map[string]interface{}, len(columns)) + for _, item := range columns { + result[item.column.Name] = item.expectedDecodedValue + } + return result +} + +func newLargeEvent4Test(t *testing.T) (*model.RowChangedEvent, *model.RowChangedEvent, *model.RowChangedEvent) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t( + t tinyint primary key, + tu1 tinyint unsigned, + tu2 tinyint unsigned, + tu3 tinyint unsigned, + tu4 tinyint unsigned, + s smallint, + su1 smallint unsigned, + su2 smallint unsigned, + su3 smallint unsigned, + su4 smallint unsigned, + m mediumint, + mu1 mediumint unsigned, + mu2 mediumint unsigned, + mu3 mediumint unsigned, + mu4 mediumint unsigned, + i int, + iu1 int unsigned, + iu2 int unsigned, + iu3 int unsigned, + iu4 int unsigned, + bi bigint, + biu1 bigint unsigned, + biu2 bigint unsigned, + biu3 bigint unsigned, + biu4 bigint unsigned, + floatT float, + doubleT double, + decimalT decimal, + floatTu float unsigned, + doubleTu double unsigned, + decimalTu decimal unsigned, + varcharT varchar(255), + charT char, + binaryT binary, + varbinaryT varbinary(255), + tinytextT tinytext, + textT text, + mediumtextT mediumtext, + longtextT longtext, + tinyblobT tinyblob, + blobT blob, + mediumblobT mediumblob, + longblobT longblob, + dateT date, + datetimeT datetime, + timestampT timestamp, + timeT time, + yearT year, + enumT enum('a', 'b', 'c'), + setT set('a', 'b', 'c'), + bitT bit(4), + jsonT json)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() - testColumns = collectAllColumns(testColumnsTable) + testColumns := collectAllColumns(testColumnsTable) - testCaseInsert = &model.RowChangedEvent{ + insert := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ - Schema: "cdc", - Table: "person", + Schema: "test", + Table: "t", }, + TableInfo: tableInfo, Columns: testColumns, PreColumns: nil, + ColInfos: colInfo, } - testCaseUpdate = &model.RowChangedEvent{ + update := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ Schema: "cdc", Table: "person", }, + TableInfo: tableInfo, Columns: testColumns, PreColumns: testColumns, + ColInfos: colInfo, } - testCaseDelete = &model.RowChangedEvent{ + deleteE := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ Schema: "cdc", Table: "person", }, + TableInfo: tableInfo, Columns: nil, PreColumns: testColumns, + ColInfos: colInfo, } - - testCaseDDL = &model.DDLEvent{ - CommitTs: 417318403368288260, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "cdc", Table: "person", - }, - }, - Query: "create table person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))", - Type: mm.ActionCreateTable, - } -) - -func collectAllColumns(groups []*testColumnTuple) []*model.Column { - result := make([]*model.Column, 0, len(groups)) - for _, item := range groups { - result = append(result, item.column) - } - return result -} - -func collectExpectedDecodedValue(columns []*testColumnTuple) map[string]interface{} { - result := make(map[string]interface{}, len(columns)) - for _, item := range columns { - result[item.column.Name] = item.expectedDecodedValue - } - return result + return insert, update, deleteE } diff --git a/cdc/sink/codec/canal/type_test.go b/cdc/sink/codec/canal/type_test.go new file mode 100644 index 00000000000..aacb5f73c00 --- /dev/null +++ b/cdc/sink/codec/canal/type_test.go @@ -0,0 +1,960 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package canal + +import ( + "testing" + + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec/common" + "github.com/pingcap/tiflow/cdc/sink/codec/internal" + "github.com/stretchr/testify/require" +) + +func TestGetMySQLType4IntTypes(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1 ( + a int primary key, + b tinyint, + c smallint, + d mediumint, + e bigint)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "int", mysqlType) + // mysql type with the default type length + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "int(11)", mysqlType) + + flag := tableInfo.ColumnsFlag[colInfos[0].ID] + javaType, err := getJavaSQLType(int64(2147483647), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(4)", mysqlType) + + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + javaType, err = getJavaSQLType(int64(127), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeTINYINT, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(6)", mysqlType) + + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + javaType, err = getJavaSQLType(int64(32767), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[3].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(9)", mysqlType) + javaType, err = getJavaSQLType(int64(8388607), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[4].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(20)", mysqlType) + javaType, err = getJavaSQLType(int64(9223372036854775807), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + + sql = `create table test.t2 ( + a int unsigned primary key, + b tinyint unsigned, + c smallint unsigned, + d mediumint unsigned, + e bigint unsigned)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[0].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "int unsigned", mysqlType) + // mysql type with the default type length + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "int(10) unsigned", mysqlType) + + javaType, err = getJavaSQLType(uint64(2147483647), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType(uint64(2147483648), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(3) unsigned", mysqlType) + + javaType, err = getJavaSQLType(uint64(127), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeTINYINT, javaType) + javaType, err = getJavaSQLType(uint64(128), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeTINYINT, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeTINYINT, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(5) unsigned", mysqlType) + javaType, err = getJavaSQLType(uint64(32767), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + javaType, err = getJavaSQLType(uint64(32768), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[3].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(8) unsigned", mysqlType) + javaType, err = getJavaSQLType(uint64(8388607), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType(uint64(8388608), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[4].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(20) unsigned", mysqlType) + javaType, err = getJavaSQLType(uint64(9223372036854775807), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + javaType, err = getJavaSQLType(uint64(9223372036854775808), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDECIMAL, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + + sql = `create table test.t3 ( + a int(10) primary key, + b tinyint(3) , + c smallint(5), + d mediumint(8), + e bigint(19))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "int", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "int(10)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(3)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(5)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(8)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(19)", mysqlType) + + sql = `create table test.t4 ( + a int(10) unsigned primary key, + b tinyint(3) unsigned, + c smallint(5) unsigned, + d mediumint(8) unsigned, + e bigint(19) unsigned)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "int unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "int(10) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(3) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(5) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(8) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(19) unsigned", mysqlType) + + sql = `create table test.t5 ( + a int zerofill primary key, + b tinyint zerofill, + c smallint unsigned zerofill, + d mediumint zerofill, + e bigint zerofill)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "int unsigned zerofill", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "int(10) unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint unsigned zerofill", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(3) unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint unsigned zerofill", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(5) unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint unsigned zerofill", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(8) unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint unsigned zerofill", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(20) unsigned zerofill", mysqlType) + + sql = `create table test.t6( + a int primary key, + b bit, + c bit(3), + d bool)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "bit", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "bit(1)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "bit", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "bit(3)", mysqlType) + javaType, err = getJavaSQLType(uint64(65), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIT, javaType) + + // bool is identical to tinyint in the TiDB. + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(1)", mysqlType) +} + +func TestGetMySQLType4FloatType(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1( + a int primary key, + b float, + c double)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "float", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "float", mysqlType) + javaType, err := getJavaSQLType(3.14, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeREAL, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "double", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "double", mysqlType) + javaType, err = getJavaSQLType(2.71, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDOUBLE, javaType) + + sql = `create table test.t2(a int primary key, b float(10, 3), c float(10))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "float", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "float(10,3)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "float", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "float", mysqlType) + + sql = `create table test.t3(a int primary key, b double(20, 3))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "double", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "double(20,3)", mysqlType) + + sql = `create table test.t4( + a int primary key, + b float unsigned, + c double unsigned, + d float zerofill, + e double zerofill)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "float unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "float unsigned", mysqlType) + javaType, err = getJavaSQLType(3.14, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeREAL, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "double unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "double unsigned", mysqlType) + javaType, err = getJavaSQLType(2.71, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDOUBLE, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "float unsigned zerofill", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "float unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "double unsigned zerofill", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "double unsigned zerofill", mysqlType) +} + +func TestGetMySQLType4Decimal(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b decimal, c numeric)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(10,0)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(10,0)", mysqlType) + + sql = `create table test.t2(a int primary key, b decimal(5), c decimal(5, 2))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(5,0)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(5,2)", mysqlType) + javaType, err := getJavaSQLType("2333", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDECIMAL, javaType) + + sql = `create table test.t3(a int primary key, b decimal unsigned, c decimal zerofill)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal unsigned", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(10,0) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal unsigned zerofill", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(10,0) unsigned zerofill", mysqlType) + javaType, err = getJavaSQLType("2333", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDECIMAL, javaType) +} + +func TestGetMySQLType4TimeTypes(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b time, c time(3))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "time", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "time", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "time", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "time(3)", mysqlType) + javaType, err := getJavaSQLType("02:20:20", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeTIME) + + sql = `create table test.t2(a int primary key, b datetime, c datetime(3))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "datetime", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "datetime", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "datetime", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "datetime(3)", mysqlType) + javaType, err = getJavaSQLType("2020-02-20 02:20:20", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeTIMESTAMP) + + sql = `create table test.t3(a int primary key, b timestamp, c timestamp(3))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "timestamp", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "timestamp", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "timestamp", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "timestamp(3)", mysqlType) + javaType, err = getJavaSQLType("2020-02-20 02:20:20", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeTIMESTAMP) + + sql = `create table test.t4(a int primary key, b date)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "date", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "date", mysqlType) + javaType, err = getJavaSQLType("2020-02-20", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeDATE) + + sql = `create table test.t5(a int primary key, b year, c year(4))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "year", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "year(4)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "year", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "year(4)", mysqlType) + javaType, err = getJavaSQLType("2020", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeVARCHAR) +} + +func TestGetMySQLType4Char(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a int primary key, b char, c char(123))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "char", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "char(1)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "char", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "char(123)", mysqlType) + javaType, err := getJavaSQLType([]uint8("测试char"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCHAR) + + sql = `create table test.t1(a int primary key, b varchar(123))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "varchar", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "varchar(123)", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试varchar"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeVARCHAR) +} + +func TestGetMySQLType4TextTypes(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b text, c tinytext, d mediumtext, e longtext)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "text", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "text", mysqlType) + javaType, err := getJavaSQLType([]uint8("测试text"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "tinytext", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "tinytext", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试tinytext"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[3].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumtext", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumtext", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试mediumtext"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[4].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "longtext", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "longtext", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试longtext"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCLOB) +} + +func TestGetMySQLType4BinaryType(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b binary, c binary(10))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "binary", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "binary(1)", mysqlType) + javaType, err := getJavaSQLType([]uint8("测试binary"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "binary", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "binary(10)", mysqlType) + + sql = `create table test.t2(a int primary key, b varbinary(23))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "varbinary", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "varbinary(23)", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试varbinary"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBLOB, javaType) +} + +func TestGetMySQLType4BlobType(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b blob, c tinyblob, d mediumblob, e longblob)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "blob", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "blob", mysqlType) + javaType, err := getJavaSQLType([]uint8("测试blob"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyblob", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyblob", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试tinyblob"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[3].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumblob", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumblob", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试mediumblob"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[4].ID] + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "longblob", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "longblob", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试longblob"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) +} + +func TestGetMySQLType4EnumAndSet(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a int primary key, b enum('a', 'b', 'c'), c set('a', 'b', 'c'))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "enum", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "enum('a','b','c')", mysqlType) + + javaType, err := getJavaSQLType(uint64(1), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + + mysqlType = common.GetMySQLType(columnInfo, false) + require.Equal(t, "set", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "set('a','b','c')", mysqlType) + + javaType, err = getJavaSQLType(uint64(2), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIT, javaType) +} + +func TestGetMySQLType4JSON(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a int primary key, b json)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := common.GetMySQLType(columnInfo, false) + require.Equal(t, "json", mysqlType) + mysqlType = common.GetMySQLType(columnInfo, true) + require.Equal(t, "json", mysqlType) + + javaType, err := getJavaSQLType("{\"key1\": \"value1\"}", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeVARCHAR, javaType) + + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeVARCHAR, javaType) +} diff --git a/cdc/sink/codec/common/config.go b/cdc/sink/codec/common/config.go index 709e23f08a8..2ce0c98d136 100644 --- a/cdc/sink/codec/common/config.go +++ b/cdc/sink/codec/common/config.go @@ -39,6 +39,7 @@ type Config struct { // canal-json only EnableTiDBExtension bool + ContentCompatible bool // avro only AvroSchemaRegistry string @@ -76,6 +77,7 @@ func NewConfig(protocol config.Protocol) *Config { const ( codecOPTEnableTiDBExtension = "enable-tidb-extension" codecOPTMaxBatchSize = "max-batch-size" + codecContentCompatible = "content-compatible" codecOPTMaxMessageBytes = "max-message-bytes" codecOPTAvroDecimalHandlingMode = "avro-decimal-handling-mode" codecOPTAvroBigintUnsignedHandlingMode = "avro-bigint-unsigned-handling-mode" @@ -120,6 +122,14 @@ func (c *Config) Apply(sinkURI *url.URL, config *config.ReplicaConfig) error { c.MaxMessageBytes = a } + if s := params.Get(codecContentCompatible); s != "" { + b, err := strconv.ParseBool(s) + if err != nil { + return err + } + c.ContentCompatible = b + } + if s := params.Get(codecOPTAvroDecimalHandlingMode); s != "" { c.AvroDecimalHandlingMode = s } diff --git a/cdc/sink/codec/common/config_test.go b/cdc/sink/codec/common/config_test.go index b8e9a761ea3..2dd584a2041 100644 --- a/cdc/sink/codec/common/config_test.go +++ b/cdc/sink/codec/common/config_test.go @@ -228,6 +228,7 @@ func TestCanalJSONHandleKeyOnly(t *testing.T) { codecConfig := NewConfig(config.ProtocolCanalJSON) err = codecConfig.Apply(sinkURI, replicaConfig) require.NoError(t, err) + require.False(t, codecConfig.ContentCompatible) err = codecConfig.Validate() require.NoError(t, err) @@ -240,6 +241,7 @@ func TestCanalJSONHandleKeyOnly(t *testing.T) { uri = "kafka://127.0.0.1:9092/large-message-handle?protocol=canal-json" sinkURI, err = url.Parse(uri) require.NoError(t, err) + require.False(t, codecConfig.ContentCompatible) codecConfig = NewConfig(config.ProtocolCanal) err = codecConfig.Apply(sinkURI, replicaConfig) @@ -329,4 +331,22 @@ func TestKafkaConfigWithoutHandleKey(t *testing.T) { err = codecConfig.Apply(sinkURI, replicaConfig) require.NoError(t, err) require.False(t, codecConfig.LargeMessageHandle.HandleKeyOnly()) + require.False(t, codecConfig.ContentCompatible) +} + +func TestCanalJSONEnableContentCompatible(t *testing.T) { + t.Parallel() + + // handle-key-only not enabled, always no error + replicaConfig := config.GetDefaultReplicaConfig() + replicaConfig.Sink.KafkaConfig = &config.KafkaConfig{} + + uri := "kafka://127.0.0.1:9092/canal-json?protocol=canal-json&content-compatible=true" + sinkURI, err := url.Parse(uri) + require.NoError(t, err) + + codecConfig := NewConfig(config.ProtocolCanalJSON) + err = codecConfig.Apply(sinkURI, replicaConfig) + require.NoError(t, err) + require.True(t, codecConfig.ContentCompatible) } diff --git a/cdc/sink/codec/common/fields_types.go b/cdc/sink/codec/common/fields_types.go new file mode 100644 index 00000000000..c8864f0f2d3 --- /dev/null +++ b/cdc/sink/codec/common/fields_types.go @@ -0,0 +1,82 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package common + +import ( + "strings" + + "github.com/pingcap/tidb/parser/charset" + timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tiflow/cdc/model" +) + +// SetBinChsClnFlag set the binary charset flag. +func SetBinChsClnFlag(ft *types.FieldType) *types.FieldType { + ft.SetCharset(charset.CharsetBin) + ft.SetCollate(charset.CollationBin) + ft.AddFlag(mysql.BinaryFlag) + return ft +} + +// SetUnsigned set the unsigned flag. +func SetUnsigned(ft *types.FieldType) *types.FieldType { + ft.SetFlag(uint(model.UnsignedFlag)) + return ft +} + +// SetElems set the elems to the ft +func SetElems(ft *types.FieldType, elems []string) *types.FieldType { + ft.SetElems(elems) + return ft +} + +// when encoding the canal format, for unsigned mysql type, add `unsigned` keyword. +// it should have the form `t unsigned`, such as `int unsigned` +func withUnsigned4MySQLType(mysqlType string, unsigned bool) string { + if unsigned && mysqlType != "bit" && mysqlType != "year" { + return mysqlType + " unsigned" + } + return mysqlType +} + +func withZerofill4MySQLType(mysqlType string, zerofill bool) string { + if zerofill && !strings.HasPrefix(mysqlType, "year") { + return mysqlType + " zerofill" + } + return mysqlType +} + +// GetMySQLType get the mysql type from column info +func GetMySQLType(columnInfo *timodel.ColumnInfo, fullType bool) string { + if !fullType { + result := types.TypeToStr(columnInfo.GetType(), columnInfo.GetCharset()) + result = withUnsigned4MySQLType(result, mysql.HasUnsignedFlag(columnInfo.GetFlag())) + result = withZerofill4MySQLType(result, mysql.HasZerofillFlag(columnInfo.GetFlag())) + return result + } + return columnInfo.GetTypeDesc() +} + +// ExtractBasicMySQLType return the mysql type +func ExtractBasicMySQLType(mysqlType string) byte { + for i := 0; i < len(mysqlType); i++ { + if mysqlType[i] == '(' || mysqlType[i] == ' ' { + return types.StrToType(mysqlType[:i]) + } + } + + return types.StrToType(mysqlType) +} diff --git a/cdc/sink/codec/internal/java.go b/cdc/sink/codec/internal/java.go index 50e06434546..cbebe37606a 100644 --- a/cdc/sink/codec/internal/java.go +++ b/cdc/sink/codec/internal/java.go @@ -115,24 +115,25 @@ func MySQLType2JavaType(mysqlType byte, isBinary bool) JavaSQLType { // Blob related is not identical to the official implementation, since we do not know `meta` at the moment. // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/dbsync/src/main/java/com/taobao/tddl/dbsync/binlog/event/RowsLogBuffer.java#L222-L231 // But this does not matter, they will be `JavaSQLTypeBlob` or `JavaSQLTypeClob` finally. - case mysql.TypeTinyBlob: - return JavaSQLTypeVARBINARY - - case mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: - return JavaSQLTypeLONGVARBINARY + case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + if isBinary { + return JavaSQLTypeBLOB + } + return JavaSQLTypeCLOB case mysql.TypeVarString, mysql.TypeVarchar: if isBinary { - return JavaSQLTypeVARBINARY + return JavaSQLTypeBLOB } return JavaSQLTypeVARCHAR case mysql.TypeString: if isBinary { - return JavaSQLTypeBINARY + return JavaSQLTypeBLOB } return JavaSQLTypeCHAR + // Geometry is not supported, this should not hit. case mysql.TypeGeometry: return JavaSQLTypeBINARY diff --git a/cdc/sinkv2/eventsink/cloudstorage/defragmenter_test.go b/cdc/sinkv2/eventsink/cloudstorage/defragmenter_test.go index ae46d92b36f..4fb389abcc2 100644 --- a/cdc/sinkv2/eventsink/cloudstorage/defragmenter_test.go +++ b/cdc/sinkv2/eventsink/cloudstorage/defragmenter_test.go @@ -20,6 +20,9 @@ import ( "testing" "time" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec/builder" "github.com/pingcap/tiflow/cdc/sinkv2/eventsink" @@ -48,7 +51,7 @@ func TestDeframenter(t *testing.T) { txnCnt := 50 sinkURI, err := url.Parse(uri) require.Nil(t, err) - encoderConfig, err := util.GetEncoderConfig(sinkURI, config.ProtocolCanalJSON, + encoderConfig, err := util.GetEncoderConfig(sinkURI, config.ProtocolCsv, config.GetDefaultReplicaConfig(), config.DefaultMaxMessageBytes) require.Nil(t, err) encoderBuilder, err := builder.NewEventBatchEncoderBuilder(ctx, encoderConfig) @@ -93,6 +96,20 @@ func TestDeframenter(t *testing.T) { {Name: "c1", Value: j + 1}, {Name: "c2", Value: "hello world"}, }, + ColInfos: []rowcodec.ColInfo{ + { + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLong), + }, + { + ID: 2, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeString), + }, + }, } frag.event.Event.Rows = append(frag.event.Event.Rows, row) encoder.AppendRowChangedEvent(ctx, "", row, nil) diff --git a/cdc/sinkv2/eventsink/cloudstorage/dml_worker_test.go b/cdc/sinkv2/eventsink/cloudstorage/dml_worker_test.go index 7556fb753f2..cb09c78060c 100644 --- a/cdc/sinkv2/eventsink/cloudstorage/dml_worker_test.go +++ b/cdc/sinkv2/eventsink/cloudstorage/dml_worker_test.go @@ -21,9 +21,7 @@ import ( "testing" "time" - timodel "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec/common" "github.com/pingcap/tiflow/cdc/sinkv2/eventsink" @@ -57,38 +55,27 @@ func testDMLWorker(ctx context.Context, t *testing.T, dir string) *dmlWorker { } func TestDMLWorkerRun(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.table1(c1 int primary key, c2 varchar(255))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfo := tableInfo.GetRowColInfos() ctx, cancel := context.WithCancel(context.Background()) parentDir := t.TempDir() d := testDMLWorker(ctx, t, parentDir) fragCh := d.inputCh - table1Dir := path.Join(parentDir, "test/table1/99") - // assume table1 and table2 are dispatched to the same DML worker - table1 := model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, - } - tableInfo := &model.TableInfo{ - TableName: model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, - }, - Version: 99, - TableInfo: &timodel.TableInfo{ - Columns: []*timodel.ColumnInfo{ - {ID: 1, Name: timodel.NewCIStr("name"), FieldType: *types.NewFieldType(mysql.TypeLong)}, - }, - }, - } + table1Dir := path.Join(parentDir, fmt.Sprintf("test/table1/%d", tableInfo.Version)) + for i := 0; i < 5; i++ { frag := eventFragment{ seqNumber: uint64(i), versionedTable: cloudstorage.VersionedTableName{ - TableNameWithPhysicTableID: table1, - TableInfoVersion: 99, + TableNameWithPhysicTableID: tableInfo.TableName, + TableInfoVersion: tableInfo.Version, }, event: &eventsink.TxnCallbackableEvent{ Event: &model.SingleTableTxn{ @@ -104,6 +91,7 @@ func TestDMLWorkerRun(t *testing.T) { {Name: "c1", Value: 100}, {Name: "c2", Value: "hello world"}, }, + ColInfos: colInfo, }, }, }, diff --git a/cdc/sinkv2/eventsink/cloudstorage/encoding_worker_test.go b/cdc/sinkv2/eventsink/cloudstorage/encoding_worker_test.go index ef56d11d327..c9bd8707795 100644 --- a/cdc/sinkv2/eventsink/cloudstorage/encoding_worker_test.go +++ b/cdc/sinkv2/eventsink/cloudstorage/encoding_worker_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/util/rowcodec" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec/builder" "github.com/pingcap/tiflow/cdc/sinkv2/eventsink" @@ -132,8 +133,6 @@ func TestEncodeEvents(t *testing.T) { } func TestEncodingWorkerRun(t *testing.T) { - t.Parallel() - encodingWorker, msgCh, encodedCh := testEncodingWorker(t) ctx, cancel := context.WithCancel(context.Background()) eg, egCtx := errgroup.WithContext(ctx) @@ -143,19 +142,27 @@ func TestEncodingWorkerRun(t *testing.T) { return defragmenter.run(egCtx) }) + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.table1(c1 int primary key, c2 varchar(255))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + table := model.TableName{ Schema: "test", Table: "table1", TableID: 100, } event := &model.SingleTableTxn{ - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, - }, + Table: &model.TableName{ + Schema: "test", + Table: "table1", + TableID: 100, }, + TableInfo: tableInfo, Rows: []*model.RowChangedEvent{ { Table: &model.TableName{ @@ -163,10 +170,12 @@ func TestEncodingWorkerRun(t *testing.T) { Table: "table1", TableID: 100, }, + TableInfo: tableInfo, Columns: []*model.Column{ {Name: "c1", Value: 100}, {Name: "c2", Value: "hello world"}, }, + ColInfos: colInfos, }, }, } diff --git a/cdc/sinkv2/eventsink/mq/mq_dml_sink_test.go b/cdc/sinkv2/eventsink/mq/mq_dml_sink_test.go index c828258a559..89ff9a0b641 100644 --- a/cdc/sinkv2/eventsink/mq/mq_dml_sink_test.go +++ b/cdc/sinkv2/eventsink/mq/mq_dml_sink_test.go @@ -21,6 +21,8 @@ import ( "time" "github.com/Shopify/sarama" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sinkv2/eventsink" "github.com/pingcap/tiflow/cdc/sinkv2/eventsink/mq/dmlproducer" @@ -76,7 +78,13 @@ func TestNewKafkaDMLSinkFailed(t *testing.T) { } func TestWriteEvents(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -101,9 +109,11 @@ func TestWriteEvents(t *testing.T) { tableStatus := state.TableSinkSinking row := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "aa"}}, + ColInfos: colInfo, } events := make([]*eventsink.CallbackableEvent[*model.SingleTableTxn], 0, 3000) diff --git a/cdc/sinkv2/eventsink/mq/worker_test.go b/cdc/sinkv2/eventsink/mq/worker_test.go index adb4e2cbfd3..af6038c6d53 100644 --- a/cdc/sinkv2/eventsink/mq/worker_test.go +++ b/cdc/sinkv2/eventsink/mq/worker_test.go @@ -19,6 +19,10 @@ import ( "testing" "time" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/rowcodec" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec/builder" "github.com/pingcap/tiflow/cdc/sink/codec/common" @@ -59,7 +63,13 @@ func newNonBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlpro } func TestNonBatchEncode_SendMessages(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -72,9 +82,11 @@ func TestNonBatchEncode_SendMessages(t *testing.T) { Partition: 1, } row := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "aa"}}, + ColInfos: colInfo, } tableStatus := state.TableSinkSinking @@ -118,7 +130,13 @@ func TestNonBatchEncode_SendMessages(t *testing.T) { } func TestBatchEncode_Batch(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -130,9 +148,11 @@ func TestBatchEncode_Batch(t *testing.T) { } tableStatus := state.TableSinkSinking row := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: 1, Value: "aa"}}, + ColInfos: colInfo, } events := make([]mqEvent, 0, 512) @@ -168,7 +188,13 @@ func TestBatchEncode_Batch(t *testing.T) { } func TestBatchEncode_Group(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() key1 := mqv1.TopicPartitionKey{ Topic: "test", @@ -193,9 +219,11 @@ func TestBatchEncode_Group(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -205,9 +233,11 @@ func TestBatchEncode_Group(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -217,9 +247,11 @@ func TestBatchEncode_Group(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 3, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + CommitTs: 3, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -229,9 +261,11 @@ func TestBatchEncode_Group(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "aa", Table: "bb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -241,9 +275,11 @@ func TestBatchEncode_Group(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "aaa", Table: "bbb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -266,7 +302,13 @@ func TestBatchEncode_Group(t *testing.T) { } func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() key1 := mqv1.TopicPartitionKey{ Topic: "test", @@ -286,9 +328,11 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: 1, Value: "aa"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &replicatingStatus, @@ -298,9 +342,11 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &replicatingStatus, @@ -310,9 +356,11 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 3, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + CommitTs: 3, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &stoppedStatus, @@ -359,7 +407,8 @@ func TestBatchEncode_SendMessages(t *testing.T) { Event: &model.RowChangedEvent{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "aa"}}, + ColInfos: []rowcodec.ColInfo{{ID: 1, Ft: types.NewFieldType(mysql.TypeVarchar)}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -371,7 +420,8 @@ func TestBatchEncode_SendMessages(t *testing.T) { Event: &model.RowChangedEvent{ CommitTs: 2, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: []rowcodec.ColInfo{{ID: 1, Ft: types.NewFieldType(mysql.TypeVarchar)}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -383,7 +433,8 @@ func TestBatchEncode_SendMessages(t *testing.T) { Event: &model.RowChangedEvent{ CommitTs: 3, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "cc"}}, + ColInfos: []rowcodec.ColInfo{{ID: 1, Ft: types.NewFieldType(mysql.TypeVarchar)}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -395,7 +446,8 @@ func TestBatchEncode_SendMessages(t *testing.T) { Event: &model.RowChangedEvent{ CommitTs: 2, Table: &model.TableName{Schema: "aa", Table: "bb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: []rowcodec.ColInfo{{ID: 1, Ft: types.NewFieldType(mysql.TypeVarchar)}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -407,7 +459,8 @@ func TestBatchEncode_SendMessages(t *testing.T) { Event: &model.RowChangedEvent{ CommitTs: 2, Table: &model.TableName{Schema: "aaa", Table: "bbb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: []rowcodec.ColInfo{{ID: 1, Ft: types.NewFieldType(mysql.TypeVarchar)}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -419,7 +472,8 @@ func TestBatchEncode_SendMessages(t *testing.T) { Event: &model.RowChangedEvent{ CommitTs: 3, Table: &model.TableName{Schema: "aaa", Table: "bbb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: []rowcodec.ColInfo{{ID: 1, Ft: types.NewFieldType(mysql.TypeVarchar)}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -477,7 +531,13 @@ func TestBatchEncodeWorker_Abort(t *testing.T) { } func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() key1 := mqv1.TopicPartitionKey{ Topic: "test", @@ -497,9 +557,11 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "aa"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &replicatingStatus, @@ -509,9 +571,11 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &replicatingStatus, @@ -521,9 +585,11 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { { rowEvent: &eventsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 3, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + CommitTs: 3, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "cc"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &stoppedStatus, diff --git a/pkg/cmd/util/helper_test.go b/pkg/cmd/util/helper_test.go index accd7c42c66..7426ac7227c 100644 --- a/pkg/cmd/util/helper_test.go +++ b/pkg/cmd/util/helper_test.go @@ -173,7 +173,7 @@ func TestStrictDecodeInvalidFile(t *testing.T) { func TestAndWriteExampleReplicaTOML(t *testing.T) { cfg := config.GetDefaultReplicaConfig() err := StrictDecodeFile("changefeed.toml", "cdc", &cfg) - require.Nil(t, err) + require.NoError(t, err) require.True(t, cfg.CaseSensitive) require.Equal(t, &config.FilterConfig{ @@ -188,7 +188,7 @@ func TestAndWriteExampleReplicaTOML(t *testing.T) { require.NoError(t, err) err = cfg.ValidateAndAdjust(sinkURL) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, &config.SinkConfig{ EncoderConcurrency: 16, DispatchRules: []*config.DispatchRule{ diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index e5e2c0e8c00..c8d606662c4 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -53,7 +53,7 @@ const ( } ], "enable-partition-separator": true, - "protocol": "open-protocol", + "protocol": "canal-json", "advance-timeout-in-sec": 150 }, "consistent": { @@ -182,7 +182,7 @@ const ( "sink": { "encoder-concurrency": 16, "dispatchers": null, - "protocol": "open-protocol", + "protocol": "canal-json", "column-selectors": [ { "matcher": [ @@ -249,7 +249,7 @@ const ( "sink": { "encoder-concurrency": 16, "dispatchers": null, - "protocol": "open-protocol", + "protocol": "canal-json", "column-selectors": [ { "matcher": [ diff --git a/pkg/config/replica_config_test.go b/pkg/config/replica_config_test.go index 63d5c7ca239..7a08b98a2c5 100644 --- a/pkg/config/replica_config_test.go +++ b/pkg/config/replica_config_test.go @@ -39,7 +39,7 @@ func TestReplicaConfigMarshal(t *testing.T) { conf.ForceReplicate = true conf.Filter.Rules = []string{"1.1"} conf.Mounter.WorkerNum = 3 - conf.Sink.Protocol = "open-protocol" + conf.Sink.Protocol = "canal-json" conf.Sink.ColumnSelectors = []*ColumnSelector{ { Matcher: []string{"1.1"}, @@ -57,7 +57,6 @@ func TestReplicaConfigMarshal(t *testing.T) { conf.Sink.DateSeparator = "month" conf.Sink.EnablePartitionSeparator = true conf.Sink.AdvanceTimeoutInSec = DefaultAdvanceTimeoutInSec - conf.Sink.KafkaConfig = &KafkaConfig{ LargeMessageHandle: &LargeMessageHandleConfig{ LargeMessageHandleOption: LargeMessageHandleOptionHandleKeyOnly, @@ -97,7 +96,7 @@ func TestReplicaConfigOutDated(t *testing.T) { conf.ForceReplicate = true conf.Filter.Rules = []string{"1.1"} conf.Mounter.WorkerNum = 3 - conf.Sink.Protocol = "open-protocol" + conf.Sink.Protocol = "canal-json" conf.Sink.DispatchRules = []*DispatchRule{ {Matcher: []string{"a.b"}, DispatcherRule: "r1"}, {Matcher: []string{"a.c"}, DispatcherRule: "r2"}, diff --git a/tests/integration_tests/canal_json_content_compatible/conf/diff_config.toml b/tests/integration_tests/canal_json_content_compatible/conf/diff_config.toml new file mode 100644 index 00000000000..7f40bf15127 --- /dev/null +++ b/tests/integration_tests/canal_json_content_compatible/conf/diff_config.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] +output-dir = "/tmp/tidb_cdc_test/canal_json_content_compatible/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/canal_json_content_compatible/data/data.sql b/tests/integration_tests/canal_json_content_compatible/data/data.sql new file mode 100644 index 00000000000..5d0ae96193a --- /dev/null +++ b/tests/integration_tests/canal_json_content_compatible/data/data.sql @@ -0,0 +1,330 @@ +drop database if exists test; +create database test; +use test; + +create table tp_int +( + id int auto_increment, + c_tinyint tinyint null, + c_smallint smallint null, + c_mediumint mediumint null, + c_int int null, + c_bigint bigint null, + constraint pk + primary key (id) +); + +insert into tp_int() +values (); + +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (1, 2, 3, 4, 5); + +-- insert max value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (127, 32767, 8388607, 2147483647, 9223372036854775807); + +-- insert min value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (-128, -32768, -8388608, -2147483648, -9223372036854775808); + +update tp_int set c_int = 0, c_tinyint = 0 where c_smallint = 2; +delete from tp_int where c_int = 0; + +-- unsigned int +create table tp_unsigned_int ( + id int auto_increment, + c_unsigned_tinyint tinyint unsigned null, + c_unsigned_smallint smallint unsigned null, + c_unsigned_mediumint mediumint unsigned null, + c_unsigned_int int unsigned null, + c_unsigned_bigint bigint unsigned null, + constraint pk + primary key (id) +); + +insert into tp_unsigned_int() +values (); + +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (1, 2, 3, 4, 5); + +-- insert max value +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (255, 65535, 16777215, 4294967295, 18446744073709551615); + +-- insert signed max value +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (127, 32767, 8388607, 2147483647, 9223372036854775807); + +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (128, 32768, 8388608, 2147483648, 9223372036854775808); + +update tp_unsigned_int set c_unsigned_int = 0, c_unsigned_tinyint = 0 where c_unsigned_smallint = 65535; +delete from tp_unsigned_int where c_unsigned_int = 0; + +-- real +create table tp_real +( + id int auto_increment, + c_float float null, + c_double double null, + c_decimal decimal null, + c_decimal_2 decimal(10, 4) null, + constraint pk + primary key (id) +); + +insert into tp_real() +values (); + +insert into tp_real(c_float, c_double, c_decimal, c_decimal_2) +values (2020.0202, 2020.0303, 2020.0404, 2021.1208); + +insert into tp_real(c_float, c_double, c_decimal, c_decimal_2) +values (-2.7182818284, -3.1415926, -8000, -179394.233); + +update tp_real set c_double = 2.333 where c_double = 2020.0303; + +-- unsigned real +create table tp_unsigned_real ( + id int auto_increment, + c_unsigned_float float unsigned null, + c_unsigned_double double unsigned null, + c_unsigned_decimal decimal unsigned null, + c_unsigned_decimal_2 decimal(10, 4) unsigned null, + constraint pk + primary key (id) +); + +insert into tp_unsigned_real() +values (); + +insert into tp_unsigned_real(c_unsigned_float, c_unsigned_double, c_unsigned_decimal, c_unsigned_decimal_2) +values (2020.0202, 2020.0303, 2020.0404, 2021.1208); + +update tp_unsigned_real set c_unsigned_double = 2020.0404 where c_unsigned_double = 2020.0303; + +-- time +create table tp_time +( + id int auto_increment, + c_date date null, + c_datetime datetime null, + c_timestamp timestamp null, + c_time time null, + c_year year null, + constraint pk + primary key (id) +); + +insert into tp_time() +values (); + +insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) +values ('2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020'); + +insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) +values ('2022-02-22', '2022-02-22 22:22:22', '2020-02-20 02:20:20', '02:20:20', '2021'); + +update tp_time set c_year = '2022' where c_year = '2020'; +update tp_time set c_date = '2022-02-22' where c_datetime = '2020-02-20 02:20:20'; + +-- text +create table tp_text +( + id int auto_increment, + c_tinytext tinytext null, + c_text text null, + c_mediumtext mediumtext null, + c_longtext longtext null, + constraint pk + primary key (id) +); + +insert into tp_text() +values (); + +insert into tp_text(c_tinytext, c_text, c_mediumtext, c_longtext) +values ('89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A'); + +insert into tp_text(c_tinytext, c_text, c_mediumtext, c_longtext) +values ('89504E470D0A1A0B', '89504E470D0A1A0B', '89504E470D0A1A0B', '89504E470D0A1A0B'); + +update tp_text set c_text = '89504E470D0A1A0B' where c_mediumtext = '89504E470D0A1A0A'; + +-- blob +create table tp_blob +( + id int auto_increment, + c_tinyblob tinyblob null, + c_blob blob null, + c_mediumblob mediumblob null, + c_longblob longblob null, + constraint pk + primary key (id) +); + +insert into tp_blob() +values (); + +insert into tp_blob(c_tinyblob, c_blob, c_mediumblob, c_longblob) +values (x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A'); + +insert into tp_blob(c_tinyblob, c_blob, c_mediumblob, c_longblob) +values (x'89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B'); + +update tp_blob set c_blob = x'89504E470D0A1A0B' where c_mediumblob = x'89504E470D0A1A0A'; + +-- char / binary +create table tp_char_binary +( + id int auto_increment, + c_char char(16) null, + c_varchar varchar(16) null, + c_binary binary(16) null, + c_varbinary varbinary(16) null, + constraint pk + primary key (id) +); + +insert into tp_char_binary() +values (); + +insert into tp_char_binary(c_char, c_varchar, c_binary, c_varbinary) +values ('89504E470D0A1A0A', '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A'); + +insert into tp_char_binary(c_char, c_varchar, c_binary, c_varbinary) +values ('89504E470D0A1A0B', '89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B'); + +update tp_char_binary set c_varchar = '89504E470D0A1A0B' where c_binary = x'89504E470D0A1A0A'; + +-- other +create table tp_other +( + id int auto_increment, + c_enum enum ('a','b','c') null, + c_set set ('a','b','c') null, + c_bit bit(64) null, + c_json json null, + constraint pk + primary key (id) +); + +insert into tp_other() +values (); + +insert into tp_other(c_enum, c_set, c_bit, c_json) +values ('a', 'a,b', b'1000001', '{ + "key1": "value1", + "key2": "value2" +}'); + +insert into tp_other(c_enum, c_set, c_bit, c_json) +values ('b', 'b,c', b'1000001', '{ + "key1": "value1", + "key2": "value2", + "key3": "123" +}'); + +update tp_other set c_enum = 'c' where c_set = 'b, c'; + +-- gbk dmls +CREATE TABLE cs_gbk ( + id INT, + name varchar(128) CHARACTER SET gbk, + country char(32) CHARACTER SET gbk, + city varchar(64), + description text CHARACTER SET gbk, + image tinyblob, + PRIMARY KEY (id) +) ENGINE = InnoDB CHARSET = utf8mb4; + +INSERT INTO cs_gbk +VALUES (1, '测试', "中国", "上海", "你好,世界" + , 0xC4E3BAC3CAC0BDE7); + +INSERT INTO cs_gbk +VALUES (2, '部署', "美国", "纽约", "世界,你好" + , 0xCAC0BDE7C4E3BAC3); + +UPDATE cs_gbk +SET name = '开发' +WHERE name = '测试'; + +DELETE FROM cs_gbk +WHERE name = '部署' + AND country = '美国' + AND city = '纽约' + AND description = '世界,你好'; + +-- ddls +CREATE TABLE test_ddl1 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +RENAME TABLE test_ddl1 TO test_ddl; + +ALTER TABLE test_ddl + ADD INDEX test_add_index (c1); + +DROP INDEX test_add_index ON test_ddl; + +ALTER TABLE test_ddl + ADD COLUMN c2 INT NOT NULL; + +TRUNCATE TABLE test_ddl; + +DROP TABLE test_ddl2; + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl3 ( + id INT, + 名称 varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +ALTER TABLE test_ddl3 + ADD COLUMN 城市 char(32); + +ALTER TABLE test_ddl3 + MODIFY COLUMN 城市 varchar(32); + +ALTER TABLE test_ddl3 + DROP COLUMN 城市; + +/* this is a DDL test for table */ +CREATE TABLE 表1 ( + id INT, + name varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +RENAME TABLE 表1 TO 表2; + +DROP TABLE 表2; + +create table finish_mark +( + id int PRIMARY KEY +); diff --git a/tests/integration_tests/canal_json_content_compatible/data/data_gbk.sql b/tests/integration_tests/canal_json_content_compatible/data/data_gbk.sql new file mode 100644 index 00000000000..504d7b41174 --- /dev/null +++ b/tests/integration_tests/canal_json_content_compatible/data/data_gbk.sql @@ -0,0 +1,101 @@ +-- This sql file is encoded in GBK by the 'iconv' command. +-- DO NOT EDIT. + +DROP DATABASE IF EXISTS test; +CREATE DATABASE test; +SET NAMES gbk; +USE test; + +-- gbk dmls +CREATE TABLE cs_gbk ( + id INT, + name varchar(128) CHARACTER SET gbk, + country char(32) CHARACTER SET gbk, + city varchar(64), + description text CHARACTER SET gbk, + image tinyblob, + PRIMARY KEY (id) +) ENGINE = InnoDB CHARSET = utf8mb4; + +INSERT INTO cs_gbk +VALUES (1, '', "й", "Ϻ", "," + , 0xC4E3BAC3CAC0BDE7); + +INSERT INTO cs_gbk +VALUES (2, '', "", "ŦԼ", "," + , 0xCAC0BDE7C4E3BAC3); + +UPDATE cs_gbk +SET name = '' +WHERE name = ''; + +DELETE FROM cs_gbk +WHERE name = '' + AND country = '' + AND city = 'ŦԼ' + AND description = ','; + +-- ddls +CREATE TABLE test_ddl1 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +RENAME TABLE test_ddl1 TO test_ddl; + +ALTER TABLE test_ddl + ADD INDEX test_add_index (c1); + +DROP INDEX test_add_index ON test_ddl; + +ALTER TABLE test_ddl + ADD COLUMN c2 INT NOT NULL; + +TRUNCATE TABLE test_ddl; + +DROP TABLE test_ddl2; + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl3 ( + id INT, + varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +ALTER TABLE test_ddl3 + ADD COLUMN char(32); + +ALTER TABLE test_ddl3 + MODIFY COLUMN varchar(32); + +ALTER TABLE test_ddl3 + DROP COLUMN ; + +/* this is a DDL test for table */ +CREATE TABLE 1 ( + id INT, + name varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +RENAME TABLE 1 TO 2; + +create table finish_mark +( + id int PRIMARY KEY +); diff --git a/tests/integration_tests/canal_json_content_compatible/run.sh b/tests/integration_tests/canal_json_content_compatible/run.sh new file mode 100644 index 00000000000..1f45432d97b --- /dev/null +++ b/tests/integration_tests/canal_json_content_compatible/run.sh @@ -0,0 +1,61 @@ +#!/bin/bash + +set -e + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +# use kafka-consumer with canal-json decoder to sync data from kafka to mysql +function run() { + if [ "$SINK_TYPE" != "kafka" ]; then + return + fi + + # clean up environment + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + # start tidb cluster + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + TOPIC_NAME="ticdc-canal-json-content-compatible" + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + if [ "$SINK_TYPE" == "kafka" ]; then + SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=canal-json&enable-tidb-extension=true&kafka-version=${KAFKA_VERSION}" + fi + + run_cdc_cli changefeed create --sink-uri="$SINK_URI" + sleep 5 # wait for changefeed to start + # determine the sink uri and run corresponding consumer + # currently only kafka and pulsar are supported + if [ "$SINK_TYPE" == "kafka" ]; then + run_kafka_consumer $WORK_DIR $SINK_URI + fi + + if [ "$SINK_TYPE" == "pulsar" ]; then + run_pulsar_consumer $WORK_DIR $SINK_URI + fi + + run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + # sync_diff can't check non-exist table, so we check expected tables are created in downstream first + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + run_sql_file $CUR/data/data_gbk.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index 21fb8d504ab..f17ca2477f7 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -15,7 +15,7 @@ mysql_only_http="http_api http_api_tls api_v2" mysql_only_consistent_replicate="consistent_replicate_ddl consistent_replicate_gbk consistent_replicate_nfs consistent_replicate_storage_file consistent_replicate_storage_file_large_value consistent_replicate_storage_s3 consistent_partition_table" kafka_only="kafka_big_messages kafka_compression kafka_messages kafka_sink_error_resume mq_sink_lost_callback" -kafka_only_protocol="canal_json_adapter_compatibility canal_json_basic multi_topics avro_basic canal_json_handle_key_only open_protocol_handle_key_only canal_json_claim_check open_protocol_claim_check" +kafka_only_protocol="canal_json_adapter_compatibility canal_json_basic canal_json_content_compatible multi_topics avro_basic canal_json_handle_key_only open_protocol_handle_key_only canal_json_claim_check open_protocol_claim_check" kafka_only_v2="kafka_big_txn_v2 kafka_big_messages_v2 multi_tables_ddl_v2 multi_topics_v2" storage_only="lossy_ddl storage_csv_update"