From 3e4522da74769120edf84d24af058c3a682675cf Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Wed, 24 Jan 2024 17:00:21 +0800 Subject: [PATCH] cdc: remove some unnecessary fields from RowChangedEvent (#10517) ref pingcap/tiflow#10386 --- cdc/entry/mounter.go | 24 +- cdc/entry/mounter_test.go | 10 +- cdc/model/codec/codec.go | 29 +- cdc/model/codec/codec_test.go | 7 +- cdc/model/schema_storage.go | 25 ++ cdc/model/schema_storage_test.go | 8 + cdc/model/sink.go | 70 ++-- cdc/model/sink_gen.go | 378 +++++++++++++++++- cdc/model/sink_gen_test.go | 113 ++++++ cdc/model/sink_test.go | 38 +- cdc/processor/sinkmanager/manager_test.go | 11 +- .../sinkmanager/redo_log_worker_test.go | 6 +- .../sinkmanager/table_sink_worker_test.go | 28 +- .../sinkmanager/table_sink_wrapper_test.go | 14 +- .../sourcemanager/sorter/mounted_iter_test.go | 9 +- cdc/redo/manager_test.go | 39 +- cdc/redo/reader/reader.go | 27 +- cdc/redo/reader/reader_test.go | 14 +- cdc/redo/writer/file/file_log_writer_test.go | 10 +- cdc/redo/writer/memory/mem_log_writer_test.go | 18 +- .../blackhole/black_hole_dml_sink_test.go | 9 +- .../cloudstorage/cloud_storage_dml_sink.go | 9 +- .../cloud_storage_dml_sink_test.go | 2 - .../dmlsink/cloudstorage/defragmenter_test.go | 11 +- .../dmlsink/cloudstorage/dml_worker_test.go | 11 +- .../cloudstorage/encoding_worker_test.go | 40 +- cdc/sink/dmlsink/event_appender.go | 8 +- cdc/sink/dmlsink/event_appender_test.go | 93 ++--- .../dmlsink/mq/dispatcher/event_router.go | 6 +- .../mq/dispatcher/event_router_test.go | 44 +- .../mq/dispatcher/partition/columns.go | 6 +- .../mq/dispatcher/partition/columns_test.go | 41 +- .../mq/dispatcher/partition/default_test.go | 9 +- .../mq/dispatcher/partition/index_value.go | 6 +- .../dispatcher/partition/index_value_test.go | 85 ++-- .../dmlsink/mq/dispatcher/partition/table.go | 4 +- .../mq/dispatcher/partition/table_test.go | 56 ++- .../mq/dispatcher/partition/ts_test.go | 48 ++- cdc/sink/dmlsink/mq/mq_dml_sink_test.go | 1 - .../columnselector/column_selector.go | 10 +- .../columnselector/column_selector_test.go | 9 +- cdc/sink/dmlsink/mq/worker_test.go | 83 ++-- cdc/sink/dmlsink/txn/event.go | 12 +- cdc/sink/dmlsink/txn/event_test.go | 68 ++-- cdc/sink/dmlsink/txn/mysql/mysql.go | 15 +- cdc/sink/dmlsink/txn/mysql/mysql_test.go | 266 ++++++++---- cdc/sink/dmlsink/txn/txn_dml_sink_test.go | 4 +- cdc/sink/tablesink/table_sink_impl_test.go | 84 ++-- cmd/kafka-consumer/main.go | 8 +- cmd/pulsar-consumer/main.go | 8 +- cmd/storage-consumer/main.go | 2 +- pkg/applier/redo.go | 2 +- pkg/applier/redo_test.go | 12 +- pkg/filter/expr_filter.go | 2 +- pkg/filter/expr_filter_bench_test.go | 8 +- pkg/filter/expr_filter_test.go | 24 +- pkg/filter/filter.go | 2 +- pkg/filter/filter_test.go | 7 +- pkg/filter/sql_event_filter.go | 2 +- pkg/filter/sql_event_filter_test.go | 8 +- pkg/sink/codec/avro/avro.go | 10 +- pkg/sink/codec/avro/avro_test.go | 11 +- pkg/sink/codec/avro/decoder.go | 8 +- pkg/sink/codec/bootstraper.go | 4 +- pkg/sink/codec/bootstraper_test.go | 16 +- pkg/sink/codec/builder/codec_test.go | 32 +- pkg/sink/codec/canal/canal_encoder_test.go | 15 +- pkg/sink/codec/canal/canal_entry.go | 2 +- pkg/sink/codec/canal/canal_entry_test.go | 30 +- pkg/sink/codec/canal/canal_json_message.go | 8 +- .../canal/canal_json_row_event_encoder.go | 16 +- .../canal_json_row_event_encoder_test.go | 20 +- .../canal/canal_json_txn_event_encoder.go | 6 +- .../canal_json_txn_event_encoder_test.go | 7 +- pkg/sink/codec/common/message_test.go | 20 +- pkg/sink/codec/craft/craft_decoder.go | 13 +- pkg/sink/codec/craft/craft_encoder_test.go | 12 +- pkg/sink/codec/craft/model.go | 12 +- pkg/sink/codec/csv/csv_encoder_test.go | 27 +- pkg/sink/codec/csv/csv_message.go | 12 +- pkg/sink/codec/csv/csv_message_test.go | 8 +- pkg/sink/codec/debezium/codec.go | 19 +- pkg/sink/codec/debezium/codec_test.go | 24 +- pkg/sink/codec/debezium/encoder.go | 4 +- pkg/sink/codec/internal/batch_tester.go | 73 ++-- .../codec/maxwell/maxwell_encoder_test.go | 10 +- pkg/sink/codec/maxwell/maxwell_message.go | 12 +- .../codec/maxwell/maxwell_message_test.go | 4 +- pkg/sink/codec/open/open_protocol_decoder.go | 4 +- pkg/sink/codec/open/open_protocol_encoder.go | 8 +- .../codec/open/open_protocol_encoder_test.go | 10 +- pkg/sink/codec/open/open_protocol_message.go | 20 +- .../codec/open/open_protocol_message_test.go | 64 +-- pkg/sink/codec/simple/avro.go | 4 +- pkg/sink/codec/simple/encoder.go | 10 +- pkg/sink/codec/simple/encoder_test.go | 22 +- pkg/sink/codec/simple/message.go | 14 +- 97 files changed, 1736 insertions(+), 858 deletions(-) diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 65a5e69eb35..4560fb1bed4 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -558,8 +558,6 @@ func (m *mounter) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntry, d } } - schemaName := tableInfo.TableName.Schema - tableName := tableInfo.TableName.Table var intRowID int64 if row.RecordID.IsInt() { intRowID = row.RecordID.IntValue() @@ -580,23 +578,17 @@ func (m *mounter) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntry, d } return &model.RowChangedEvent{ - StartTs: row.StartTs, - CommitTs: row.CRTs, - RowID: intRowID, - Table: &model.TableName{ - Schema: schemaName, - Table: tableName, - TableID: row.PhysicalTableID, - IsPartition: tableInfo.GetPartitionInfo() != nil, - }, - ColInfos: extendColumnInfos, - TableInfo: tableInfo, - Columns: cols, - PreColumns: preCols, + StartTs: row.StartTs, + CommitTs: row.CRTs, + RowID: intRowID, + PhysicalTableID: row.PhysicalTableID, + ColInfos: extendColumnInfos, + TableInfo: tableInfo, + Columns: cols, + PreColumns: preCols, Checksum: checksum, - IndexColumns: tableInfo.IndexColumnsOffset, ApproximateDataSize: dataSize, }, rawRow, nil } diff --git a/cdc/entry/mounter_test.go b/cdc/entry/mounter_test.go index db0d1ec65cd..45ad3adea86 100644 --- a/cdc/entry/mounter_test.go +++ b/cdc/entry/mounter_test.go @@ -331,8 +331,8 @@ func testMounterDisableOldValue(t *testing.T, tc struct { return } rows++ - require.Equal(t, row.Table.Table, tc.tableName) - require.Equal(t, row.Table.Schema, "test") + require.Equal(t, row.TableInfo.GetTableName(), tc.tableName) + require.Equal(t, row.TableInfo.GetSchemaName(), "test") // [TODO] check size and reopen this check // require.Equal(t, rowBytes[rows-1], row.ApproximateBytes(), row) t.Log("ApproximateBytes", tc.tableName, rows-1, row.ApproximateBytes()) @@ -1491,10 +1491,10 @@ func TestDecodeEventIgnoreRow(t *testing.T) { } row := pEvent.Row rows++ - require.Equal(t, row.Table.Schema, "test") + require.Equal(t, row.TableInfo.GetSchemaName(), "test") // Now we only allow filter dml event by table, so we only check row's table. - require.NotContains(t, ignoredTables, row.Table.Table) - require.Contains(t, tables, row.Table.Table) + require.NotContains(t, ignoredTables, row.TableInfo.GetTableName()) + require.Contains(t, tables, row.TableInfo.GetTableName()) }) return rows } diff --git a/cdc/model/codec/codec.go b/cdc/model/codec/codec.go index 43a6c41243d..7aeb5bad5b1 100644 --- a/cdc/model/codec/codec.go +++ b/cdc/model/codec/codec.go @@ -161,15 +161,6 @@ func MarshalRedoLog(r *model.RedoLog, b []byte) (o []byte, err error) { return } -// MarshalRowAsRedoLog converts a RowChangedEvent into RedoLog, and then marshals it. -func MarshalRowAsRedoLog(r *model.RowChangedEvent, b []byte) (o []byte, err error) { - log := &model.RedoLog{ - RedoRow: model.RedoRowChangedEvent{Row: r}, - Type: model.RedoLogTypeRow, - } - return MarshalRedoLog(log, b) -} - // MarshalDDLAsRedoLog converts a DDLEvent into RedoLog, and then marshals it. func MarshalDDLAsRedoLog(d *model.DDLEvent, b []byte) (o []byte, err error) { log := &model.RedoLog{ @@ -187,19 +178,13 @@ func decodeVersion(bts []byte) (uint16, []byte) { func redoLogFromV1(rv1 *codecv1.RedoLog) (r *model.RedoLog) { r = &model.RedoLog{Type: (model.RedoLogType)(rv1.Type)} if rv1.RedoRow != nil && rv1.RedoRow.Row != nil { - r.RedoRow.Row = &model.RowChangedEvent{ - StartTs: rv1.RedoRow.Row.StartTs, - CommitTs: rv1.RedoRow.Row.CommitTs, - RowID: rv1.RedoRow.Row.RowID, - Table: tableNameFromV1(rv1.RedoRow.Row.Table), - ColInfos: rv1.RedoRow.Row.ColInfos, - TableInfo: rv1.RedoRow.Row.TableInfo, - Columns: make([]*model.Column, 0, len(rv1.RedoRow.Row.Columns)), - PreColumns: make([]*model.Column, 0, len(rv1.RedoRow.Row.PreColumns)), - IndexColumns: rv1.RedoRow.Row.IndexColumns, - ApproximateDataSize: rv1.RedoRow.Row.ApproximateDataSize, - SplitTxn: rv1.RedoRow.Row.SplitTxn, - ReplicatingTs: rv1.RedoRow.Row.ReplicatingTs, + r.RedoRow.Row = &model.RowChangedEventInRedoLog{ + StartTs: rv1.RedoRow.Row.StartTs, + CommitTs: rv1.RedoRow.Row.CommitTs, + Table: tableNameFromV1(rv1.RedoRow.Row.Table), + Columns: make([]*model.Column, 0, len(rv1.RedoRow.Row.Columns)), + PreColumns: make([]*model.Column, 0, len(rv1.RedoRow.Row.PreColumns)), + IndexColumns: rv1.RedoRow.Row.IndexColumns, } for _, c := range rv1.RedoRow.Row.Columns { r.RedoRow.Row.Columns = append(r.RedoRow.Row.Columns, columnFromV1(c)) diff --git a/cdc/model/codec/codec_test.go b/cdc/model/codec/codec_test.go index 5dff63c25d0..03577cbcdf3 100644 --- a/cdc/model/codec/codec_test.go +++ b/cdc/model/codec/codec_test.go @@ -68,7 +68,7 @@ func TestV1toV2(t *testing.T) { rv2 = &model.RedoLog{ RedoRow: model.RedoRowChangedEvent{ - Row: &model.RowChangedEvent{ + Row: &model.RowChangedEventInRedoLog{ StartTs: 1, CommitTs: 2, Table: &model.TableName{ @@ -77,7 +77,6 @@ func TestV1toV2(t *testing.T) { TableID: 1, IsPartition: false, }, - TableInfo: nil, Columns: []*model.Column{ { Name: "column", @@ -136,7 +135,7 @@ func TestV1toV2(t *testing.T) { func TestRowRedoConvert(t *testing.T) { t.Parallel() - row := &model.RowChangedEvent{ + row := &model.RowChangedEventInRedoLog{ StartTs: 100, CommitTs: 120, Table: &model.TableName{Schema: "test", Table: "table1", TableID: 57}, @@ -196,7 +195,7 @@ func TestRowRedoConvert(t *testing.T) { func TestRowRedoConvertWithEmptySlice(t *testing.T) { t.Parallel() - row := &model.RowChangedEvent{ + row := &model.RowChangedEventInRedoLog{ StartTs: 100, CommitTs: 120, Table: &model.TableName{Schema: "test", Table: "table1", TableID: 57}, diff --git a/cdc/model/schema_storage.go b/cdc/model/schema_storage.go index e8be9b8210a..95c6e971cff 100644 --- a/cdc/model/schema_storage.go +++ b/cdc/model/schema_storage.go @@ -315,6 +315,31 @@ func (ti *TableInfo) ForceGetColumnIDByName(name string) int64 { return colID } +// GetSchemaName returns the schema name of the table +func (ti *TableInfo) GetSchemaName() string { + return ti.TableName.Schema +} + +// GetTableName returns the table name of the table +func (ti *TableInfo) GetTableName() string { + return ti.TableName.Table +} + +// GetSchemaNamePtr returns the pointer to the schema name of the table +func (ti *TableInfo) GetSchemaNamePtr() *string { + return &ti.TableName.Schema +} + +// GetTableNamePtr returns the pointer to the table name of the table +func (ti *TableInfo) GetTableNamePtr() *string { + return &ti.TableName.Table +} + +// IsPartitionTable returns whether the table is partition table +func (ti *TableInfo) IsPartitionTable() bool { + return ti.TableName.IsPartition +} + func (ti *TableInfo) String() string { return fmt.Sprintf("TableInfo, ID: %d, Name:%s, ColNum: %d, IdxNum: %d, PKIsHandle: %t", ti.ID, ti.TableName, len(ti.Columns), len(ti.Indices), ti.PKIsHandle) } diff --git a/cdc/model/schema_storage_test.go b/cdc/model/schema_storage_test.go index 212afc3b8c3..64fd4f53586 100644 --- a/cdc/model/schema_storage_test.go +++ b/cdc/model/schema_storage_test.go @@ -16,6 +16,7 @@ package model import ( "testing" + "github.com/pingcap/tidb/pkg/parser/model" timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" parser_types "github.com/pingcap/tidb/pkg/parser/types" @@ -174,6 +175,9 @@ func TestTableInfoGetterFuncs(t *testing.T) { }, IsCommonHandle: false, PKIsHandle: false, + Partition: &model.PartitionInfo{ + Enable: true, + }, } info := WrapTableInfo(1, "test", 0, &tbl) @@ -185,6 +189,10 @@ func TestTableInfoGetterFuncs(t *testing.T) { require.Equal(t, "TableInfo, ID: 1071, Name:test.t1, ColNum: 3, IdxNum: 1, PKIsHandle: false", info.String()) + require.Equal(t, "test", info.GetSchemaName()) + require.Equal(t, "t1", info.GetTableName()) + require.True(t, info.IsPartitionTable()) + handleColIDs, fts, colInfos := info.GetRowColInfos() require.Equal(t, []int64{-1}, handleColIDs) require.Equal(t, 3, len(fts)) diff --git a/cdc/model/sink.go b/cdc/model/sink.go index f21ee99a02d..76cd739431b 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -277,9 +277,9 @@ func (r *RedoLog) TrySplitAndSortUpdateEvent(_ string) error { // RedoRowChangedEvent represents the DML event used in RedoLog type RedoRowChangedEvent struct { - Row *RowChangedEvent `msg:"row"` - Columns []RedoColumn `msg:"columns"` - PreColumns []RedoColumn `msg:"pre-columns"` + Row *RowChangedEventInRedoLog `msg:"row"` + Columns []RedoColumn `msg:"columns"` + PreColumns []RedoColumn `msg:"pre-columns"` } // RedoDDLEvent represents DDL event used in redo log persistent @@ -291,8 +291,21 @@ type RedoDDLEvent struct { // ToRedoLog converts row changed event to redo log func (r *RowChangedEvent) ToRedoLog() *RedoLog { + rowInRedoLog := &RowChangedEventInRedoLog{ + StartTs: r.StartTs, + CommitTs: r.CommitTs, + Table: &TableName{ + Schema: r.TableInfo.GetSchemaName(), + Table: r.TableInfo.GetTableName(), + TableID: r.PhysicalTableID, + IsPartition: r.TableInfo.IsPartitionTable(), + }, + Columns: r.Columns, + PreColumns: r.PreColumns, + IndexColumns: r.TableInfo.IndexColumnsOffset, + } return &RedoLog{ - RedoRow: RedoRowChangedEvent{Row: r}, + RedoRow: RedoRowChangedEvent{Row: rowInRedoLog}, Type: RedoLogTypeRow, } } @@ -312,10 +325,10 @@ type RowChangedEvent struct { RowID int64 `json:"row-id" msg:"-"` // Deprecated. It is empty when the RowID comes from clustered index table. - // Table contains the table name and table ID. - // NOTICE: We store the physical table ID here, not the logical table ID. - Table *TableName `json:"table" msg:"table"` + PhysicalTableID int64 `json:"physical-tbl-id" msg:"physical-tbl-id"` + ColInfos []rowcodec.ColInfo `json:"column-infos" msg:"-"` + // NOTICE: We probably store the logical ID inside TableInfo's TableName, // not the physical ID. // For normal table, there is only one ID, which is the physical ID. @@ -328,9 +341,8 @@ type RowChangedEvent struct { // So be careful when using the TableInfo. TableInfo *TableInfo `json:"-" msg:"-"` - Columns []*Column `json:"columns" msg:"columns"` - PreColumns []*Column `json:"pre-columns" msg:"pre-columns"` - IndexColumns [][]int `json:"-" msg:"index-columns"` + Columns []*Column `json:"columns" msg:"columns"` + PreColumns []*Column `json:"pre-columns" msg:"pre-columns"` // Checksum for the event, only not nil if the upstream TiDB enable the row level checksum // and TiCDC set the integrity check level to the correctness. @@ -346,6 +358,20 @@ type RowChangedEvent struct { ReplicatingTs Ts `json:"-" msg:"-"` } +// RowChangedEventInRedoLog is used to store RowChangedEvent in redo log v2 format +type RowChangedEventInRedoLog struct { + StartTs uint64 `json:"start-ts" msg:"start-ts"` + CommitTs uint64 `json:"commit-ts" msg:"commit-ts"` + + // Table contains the table name and table ID. + // NOTICE: We store the physical table ID here, not the logical table ID. + Table *TableName `json:"table" msg:"table"` + + Columns []*Column `json:"columns" msg:"columns"` + PreColumns []*Column `json:"pre-columns" msg:"pre-columns"` + IndexColumns [][]int `json:"-" msg:"index-columns"` +} + // txnRows represents a set of events that belong to the same transaction. type txnRows []*RowChangedEvent @@ -480,12 +506,8 @@ func (r *RowChangedEvent) WithHandlePrimaryFlag(colNames map[string]struct{}) { // ApproximateBytes returns approximate bytes in memory consumed by the event. func (r *RowChangedEvent) ApproximateBytes() int { const sizeOfRowEvent = int(unsafe.Sizeof(*r)) - const sizeOfTable = int(unsafe.Sizeof(*r.Table)) - const sizeOfIndexes = int(unsafe.Sizeof(r.IndexColumns[0])) - const sizeOfInt = int(unsafe.Sizeof(int(0))) - // Size of table name - size := len(r.Table.Schema) + len(r.Table.Table) + sizeOfTable + size := 0 // Size of cols for i := range r.Columns { size += r.Columns[i].ApproximateBytes @@ -496,11 +518,6 @@ func (r *RowChangedEvent) ApproximateBytes() int { size += r.PreColumns[i].ApproximateBytes } } - // Size of index columns - for i := range r.IndexColumns { - size += len(r.IndexColumns[i]) * sizeOfInt - size += sizeOfIndexes - } // Size of an empty row event size += sizeOfRowEvent return size @@ -858,8 +875,8 @@ func NewBootstrapDDLEvent(tableInfo *TableInfo) *DDLEvent { // //msgp:ignore SingleTableTxn type SingleTableTxn struct { - Table *TableName - TableInfo *TableInfo + PhysicalTableID int64 + TableInfo *TableInfo // TableInfoVersion is the version of the table info, it is used to generate data path // in storage sink. Generally, TableInfoVersion equals to `SingleTableTxn.TableInfo.Version`. // Besides, if one table is just scheduled to a new processor, the TableInfoVersion should be @@ -876,6 +893,11 @@ func (t *SingleTableTxn) GetCommitTs() uint64 { return t.CommitTs } +// GetPhysicalTableID returns the physical table id of the table in the transaction +func (t *SingleTableTxn) GetPhysicalTableID() int64 { + return t.PhysicalTableID +} + // TrySplitAndSortUpdateEvent split update events if unique key is updated func (t *SingleTableTxn) TrySplitAndSortUpdateEvent(scheme string) error { if !t.shouldSplitUpdateEvent(scheme) { @@ -1000,11 +1022,11 @@ func splitUpdateEvent( // Append adds a row changed event into SingleTableTxn func (t *SingleTableTxn) Append(row *RowChangedEvent) { - if row.StartTs != t.StartTs || row.CommitTs != t.CommitTs || row.Table.TableID != t.Table.TableID { + if row.StartTs != t.StartTs || row.CommitTs != t.CommitTs || row.PhysicalTableID != t.GetPhysicalTableID() { log.Panic("unexpected row change event", zap.Uint64("startTs", t.StartTs), zap.Uint64("commitTs", t.CommitTs), - zap.Any("table", t.Table), + zap.Any("table", t.GetPhysicalTableID()), zap.Any("row", row)) } t.Rows = append(t.Rows, row) diff --git a/cdc/model/sink_gen.go b/cdc/model/sink_gen.go index 8760f795c1e..e601897b66d 100644 --- a/cdc/model/sink_gen.go +++ b/cdc/model/sink_gen.go @@ -1097,7 +1097,7 @@ func (z *RedoRowChangedEvent) DecodeMsg(dc *msgp.Reader) (err error) { z.Row = nil } else { if z.Row == nil { - z.Row = new(RowChangedEvent) + z.Row = new(RowChangedEventInRedoLog) } err = z.Row.DecodeMsg(dc) if err != nil { @@ -1416,7 +1416,7 @@ func (z *RedoRowChangedEvent) UnmarshalMsg(bts []byte) (o []byte, err error) { z.Row = nil } else { if z.Row == nil { - z.Row = new(RowChangedEvent) + z.Row = new(RowChangedEventInRedoLog) } bts, err = z.Row.UnmarshalMsg(bts) if err != nil { @@ -1565,6 +1565,372 @@ func (z *RedoRowChangedEvent) Msgsize() (s int) { // DecodeMsg implements msgp.Decodable func (z *RowChangedEvent) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "start-ts": + z.StartTs, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "StartTs") + return + } + case "commit-ts": + z.CommitTs, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "CommitTs") + return + } + case "physical-tbl-id": + z.PhysicalTableID, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "PhysicalTableID") + return + } + case "columns": + var zb0002 uint32 + zb0002, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err, "Columns") + return + } + if cap(z.Columns) >= int(zb0002) { + z.Columns = (z.Columns)[:zb0002] + } else { + z.Columns = make([]*Column, zb0002) + } + for za0001 := range z.Columns { + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "Columns", za0001) + return + } + z.Columns[za0001] = nil + } else { + if z.Columns[za0001] == nil { + z.Columns[za0001] = new(Column) + } + err = z.Columns[za0001].DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Columns", za0001) + return + } + } + } + case "pre-columns": + var zb0003 uint32 + zb0003, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err, "PreColumns") + return + } + if cap(z.PreColumns) >= int(zb0003) { + z.PreColumns = (z.PreColumns)[:zb0003] + } else { + z.PreColumns = make([]*Column, zb0003) + } + for za0002 := range z.PreColumns { + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "PreColumns", za0002) + return + } + z.PreColumns[za0002] = nil + } else { + if z.PreColumns[za0002] == nil { + z.PreColumns[za0002] = new(Column) + } + err = z.PreColumns[za0002].DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "PreColumns", za0002) + return + } + } + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *RowChangedEvent) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 5 + // write "start-ts" + err = en.Append(0x85, 0xa8, 0x73, 0x74, 0x61, 0x72, 0x74, 0x2d, 0x74, 0x73) + if err != nil { + return + } + err = en.WriteUint64(z.StartTs) + if err != nil { + err = msgp.WrapError(err, "StartTs") + return + } + // write "commit-ts" + err = en.Append(0xa9, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x2d, 0x74, 0x73) + if err != nil { + return + } + err = en.WriteUint64(z.CommitTs) + if err != nil { + err = msgp.WrapError(err, "CommitTs") + return + } + // write "physical-tbl-id" + err = en.Append(0xaf, 0x70, 0x68, 0x79, 0x73, 0x69, 0x63, 0x61, 0x6c, 0x2d, 0x74, 0x62, 0x6c, 0x2d, 0x69, 0x64) + if err != nil { + return + } + err = en.WriteInt64(z.PhysicalTableID) + if err != nil { + err = msgp.WrapError(err, "PhysicalTableID") + return + } + // write "columns" + err = en.Append(0xa7, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73) + if err != nil { + return + } + err = en.WriteArrayHeader(uint32(len(z.Columns))) + if err != nil { + err = msgp.WrapError(err, "Columns") + return + } + for za0001 := range z.Columns { + if z.Columns[za0001] == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = z.Columns[za0001].EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Columns", za0001) + return + } + } + } + // write "pre-columns" + err = en.Append(0xab, 0x70, 0x72, 0x65, 0x2d, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73) + if err != nil { + return + } + err = en.WriteArrayHeader(uint32(len(z.PreColumns))) + if err != nil { + err = msgp.WrapError(err, "PreColumns") + return + } + for za0002 := range z.PreColumns { + if z.PreColumns[za0002] == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = z.PreColumns[za0002].EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "PreColumns", za0002) + return + } + } + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *RowChangedEvent) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 5 + // string "start-ts" + o = append(o, 0x85, 0xa8, 0x73, 0x74, 0x61, 0x72, 0x74, 0x2d, 0x74, 0x73) + o = msgp.AppendUint64(o, z.StartTs) + // string "commit-ts" + o = append(o, 0xa9, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x2d, 0x74, 0x73) + o = msgp.AppendUint64(o, z.CommitTs) + // string "physical-tbl-id" + o = append(o, 0xaf, 0x70, 0x68, 0x79, 0x73, 0x69, 0x63, 0x61, 0x6c, 0x2d, 0x74, 0x62, 0x6c, 0x2d, 0x69, 0x64) + o = msgp.AppendInt64(o, z.PhysicalTableID) + // string "columns" + o = append(o, 0xa7, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73) + o = msgp.AppendArrayHeader(o, uint32(len(z.Columns))) + for za0001 := range z.Columns { + if z.Columns[za0001] == nil { + o = msgp.AppendNil(o) + } else { + o, err = z.Columns[za0001].MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Columns", za0001) + return + } + } + } + // string "pre-columns" + o = append(o, 0xab, 0x70, 0x72, 0x65, 0x2d, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73) + o = msgp.AppendArrayHeader(o, uint32(len(z.PreColumns))) + for za0002 := range z.PreColumns { + if z.PreColumns[za0002] == nil { + o = msgp.AppendNil(o) + } else { + o, err = z.PreColumns[za0002].MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "PreColumns", za0002) + return + } + } + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *RowChangedEvent) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "start-ts": + z.StartTs, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "StartTs") + return + } + case "commit-ts": + z.CommitTs, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "CommitTs") + return + } + case "physical-tbl-id": + z.PhysicalTableID, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "PhysicalTableID") + return + } + case "columns": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Columns") + return + } + if cap(z.Columns) >= int(zb0002) { + z.Columns = (z.Columns)[:zb0002] + } else { + z.Columns = make([]*Column, zb0002) + } + for za0001 := range z.Columns { + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + z.Columns[za0001] = nil + } else { + if z.Columns[za0001] == nil { + z.Columns[za0001] = new(Column) + } + bts, err = z.Columns[za0001].UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Columns", za0001) + return + } + } + } + case "pre-columns": + var zb0003 uint32 + zb0003, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "PreColumns") + return + } + if cap(z.PreColumns) >= int(zb0003) { + z.PreColumns = (z.PreColumns)[:zb0003] + } else { + z.PreColumns = make([]*Column, zb0003) + } + for za0002 := range z.PreColumns { + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + z.PreColumns[za0002] = nil + } else { + if z.PreColumns[za0002] == nil { + z.PreColumns[za0002] = new(Column) + } + bts, err = z.PreColumns[za0002].UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "PreColumns", za0002) + return + } + } + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *RowChangedEvent) Msgsize() (s int) { + s = 1 + 9 + msgp.Uint64Size + 10 + msgp.Uint64Size + 16 + msgp.Int64Size + 8 + msgp.ArrayHeaderSize + for za0001 := range z.Columns { + if z.Columns[za0001] == nil { + s += msgp.NilSize + } else { + s += z.Columns[za0001].Msgsize() + } + } + s += 12 + msgp.ArrayHeaderSize + for za0002 := range z.PreColumns { + if z.PreColumns[za0002] == nil { + s += msgp.NilSize + } else { + s += z.PreColumns[za0002].Msgsize() + } + } + return +} + +// DecodeMsg implements msgp.Decodable +func (z *RowChangedEventInRedoLog) DecodeMsg(dc *msgp.Reader) (err error) { var field []byte _ = field var zb0001 uint32 @@ -1717,7 +2083,7 @@ func (z *RowChangedEvent) DecodeMsg(dc *msgp.Reader) (err error) { } // EncodeMsg implements msgp.Encodable -func (z *RowChangedEvent) EncodeMsg(en *msgp.Writer) (err error) { +func (z *RowChangedEventInRedoLog) EncodeMsg(en *msgp.Writer) (err error) { // map header, size 6 // write "start-ts" err = en.Append(0x86, 0xa8, 0x73, 0x74, 0x61, 0x72, 0x74, 0x2d, 0x74, 0x73) @@ -1832,7 +2198,7 @@ func (z *RowChangedEvent) EncodeMsg(en *msgp.Writer) (err error) { } // MarshalMsg implements msgp.Marshaler -func (z *RowChangedEvent) MarshalMsg(b []byte) (o []byte, err error) { +func (z *RowChangedEventInRedoLog) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) // map header, size 6 // string "start-ts" @@ -1893,7 +2259,7 @@ func (z *RowChangedEvent) MarshalMsg(b []byte) (o []byte, err error) { } // UnmarshalMsg implements msgp.Unmarshaler -func (z *RowChangedEvent) UnmarshalMsg(bts []byte) (o []byte, err error) { +func (z *RowChangedEventInRedoLog) UnmarshalMsg(bts []byte) (o []byte, err error) { var field []byte _ = field var zb0001 uint32 @@ -2044,7 +2410,7 @@ func (z *RowChangedEvent) UnmarshalMsg(bts []byte) (o []byte, err error) { } // Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message -func (z *RowChangedEvent) Msgsize() (s int) { +func (z *RowChangedEventInRedoLog) Msgsize() (s int) { s = 1 + 9 + msgp.Uint64Size + 10 + msgp.Uint64Size + 6 if z.Table == nil { s += msgp.NilSize diff --git a/cdc/model/sink_gen_test.go b/cdc/model/sink_gen_test.go index 43963ef2e0b..8235d6196af 100644 --- a/cdc/model/sink_gen_test.go +++ b/cdc/model/sink_gen_test.go @@ -800,6 +800,119 @@ func BenchmarkDecodeRowChangedEvent(b *testing.B) { } } +func TestMarshalUnmarshalRowChangedEventInRedoLog(t *testing.T) { + v := RowChangedEventInRedoLog{} + bts, err := v.MarshalMsg(nil) + if err != nil { + t.Fatal(err) + } + left, err := v.UnmarshalMsg(bts) + if err != nil { + t.Fatal(err) + } + if len(left) > 0 { + t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left) + } + + left, err = msgp.Skip(bts) + if err != nil { + t.Fatal(err) + } + if len(left) > 0 { + t.Errorf("%d bytes left over after Skip(): %q", len(left), left) + } +} + +func BenchmarkMarshalMsgRowChangedEventInRedoLog(b *testing.B) { + v := RowChangedEventInRedoLog{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgRowChangedEventInRedoLog(b *testing.B) { + v := RowChangedEventInRedoLog{} + bts := make([]byte, 0, v.Msgsize()) + bts, _ = v.MarshalMsg(bts[0:0]) + b.SetBytes(int64(len(bts))) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + bts, _ = v.MarshalMsg(bts[0:0]) + } +} + +func BenchmarkUnmarshalRowChangedEventInRedoLog(b *testing.B) { + v := RowChangedEventInRedoLog{} + bts, _ := v.MarshalMsg(nil) + b.ReportAllocs() + b.SetBytes(int64(len(bts))) + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := v.UnmarshalMsg(bts) + if err != nil { + b.Fatal(err) + } + } +} + +func TestEncodeDecodeRowChangedEventInRedoLog(t *testing.T) { + v := RowChangedEventInRedoLog{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeRowChangedEventInRedoLog Msgsize() is inaccurate") + } + + vn := RowChangedEventInRedoLog{} + err := msgp.Decode(&buf, &vn) + if err != nil { + t.Error(err) + } + + buf.Reset() + msgp.Encode(&buf, &v) + err = msgp.NewReader(&buf).Skip() + if err != nil { + t.Error(err) + } +} + +func BenchmarkEncodeRowChangedEventInRedoLog(b *testing.B) { + v := RowChangedEventInRedoLog{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + b.SetBytes(int64(buf.Len())) + en := msgp.NewWriter(msgp.Nowhere) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.EncodeMsg(en) + } + en.Flush() +} + +func BenchmarkDecodeRowChangedEventInRedoLog(b *testing.B) { + v := RowChangedEventInRedoLog{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + b.SetBytes(int64(buf.Len())) + rd := msgp.NewEndlessReader(buf.Bytes(), b) + dc := msgp.NewReader(rd) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + err := v.DecodeMsg(dc) + if err != nil { + b.Fatal(err) + } + } +} + func TestMarshalUnmarshalTableName(t *testing.T) { v := TableName{} bts, err := v.MarshalMsg(nil) diff --git a/cdc/model/sink_test.go b/cdc/model/sink_test.go index 4089db24525..dc75ae8b32d 100644 --- a/cdc/model/sink_test.go +++ b/cdc/model/sink_test.go @@ -89,9 +89,11 @@ func TestTableNameFuncs(t *testing.T) { func TestRowChangedEventFuncs(t *testing.T) { t.Parallel() deleteRow := &RowChangedEvent{ - Table: &TableName{ - Schema: "test", - Table: "t1", + TableInfo: &TableInfo{ + TableName: TableName{ + Schema: "test", + Table: "t1", + }, }, PreColumns: []*Column{ { @@ -609,3 +611,33 @@ func TestTrySplitAndSortUpdateEventOne(t *testing.T) { require.NoError(t, err) require.Len(t, txn.Rows, 1) } + +func TestToRedoLog(t *testing.T) { + event := &RowChangedEvent{ + StartTs: 100, + CommitTs: 1000, + PhysicalTableID: 1, + TableInfo: &TableInfo{ + TableName: TableName{Schema: "test", Table: "t"}, + }, + Columns: []*Column{ + { + Name: "col1", + Flag: BinaryFlag, + Value: "col1-value", + }, + { + Name: "col2", + Flag: HandleKeyFlag | UniqueKeyFlag, + Value: "col2-value-updated", + }, + }, + } + eventInRedoLog := event.ToRedoLog() + require.Equal(t, event.StartTs, eventInRedoLog.RedoRow.Row.StartTs) + require.Equal(t, event.CommitTs, eventInRedoLog.RedoRow.Row.CommitTs) + require.Equal(t, event.PhysicalTableID, eventInRedoLog.RedoRow.Row.Table.TableID) + require.Equal(t, event.TableInfo.GetSchemaName(), eventInRedoLog.RedoRow.Row.Table.Schema) + require.Equal(t, event.TableInfo.GetTableName(), eventInRedoLog.RedoRow.Row.Table.Table) + require.Equal(t, event.Columns, eventInRedoLog.RedoRow.Row.Columns) +} diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index e51af2a3f13..10e198e2d72 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -47,7 +47,7 @@ func addTableAndAddEventsToSortEngine( t *testing.T, engine sorter.SortEngine, span tablepb.Span, -) { +) uint64 { engine.AddTable(span, 0) events := []*model.PolymorphicEvent{ { @@ -105,9 +105,14 @@ func addTableAndAddEventsToSortEngine( }, }, } + size := uint64(0) for _, event := range events { + if event.Row != nil { + size += uint64(event.Row.ApproximateBytes()) + } engine.Add(span, event) } + return size } func TestAddTable(t *testing.T) { @@ -157,13 +162,13 @@ func TestRemoveTable(t *testing.T) { require.NotNil(t, tableSink) err := manager.StartTable(span, 0) require.NoError(t, err) - addTableAndAddEventsToSortEngine(t, e, span) + totalEventSize := addTableAndAddEventsToSortEngine(t, e, span) manager.UpdateBarrierTs(4, nil) manager.UpdateReceivedSorterResolvedTs(span, 5) manager.schemaStorage.AdvanceResolvedTs(5) // Check all the events are sent to sink and record the memory usage. require.Eventually(t, func() bool { - return manager.sinkMemQuota.GetUsedBytes() == 904 + return manager.sinkMemQuota.GetUsedBytes() == totalEventSize }, 5*time.Second, 10*time.Millisecond) // Call this function times to test the idempotence. diff --git a/cdc/processor/sinkmanager/redo_log_worker_test.go b/cdc/processor/sinkmanager/redo_log_worker_test.go index fdddc1faf95..14ca37b25bd 100644 --- a/cdc/processor/sinkmanager/redo_log_worker_test.go +++ b/cdc/processor/sinkmanager/redo_log_worker_test.go @@ -39,10 +39,10 @@ type redoLogWorkerSuite struct { } func (suite *redoLogWorkerSuite) SetupSuite() { - requestMemSize = testEventSize + requestMemSize = uint64(testEventSize) // For one batch size. // Advance table sink per 2 events. - maxUpdateIntervalSize = testEventSize * 2 + maxUpdateIntervalSize = uint64(testEventSize * 2) suite.testChangefeedID = model.DefaultChangeFeedID("1") suite.testSpan = spanz.TableIDToComparableSpan(1) } @@ -69,7 +69,7 @@ func (suite *redoLogWorkerSuite) createWorker( // To avoid refund or release panics. quota := memquota.NewMemQuota(suite.testChangefeedID, memQuota, "sink") // NOTICE: Do not forget the initial memory quota in the worker first time running. - quota.ForceAcquire(testEventSize) + quota.ForceAcquire(uint64(testEventSize)) quota.AddTable(suite.testSpan) redoDMLManager := newMockRedoDMLManager() diff --git a/cdc/processor/sinkmanager/table_sink_worker_test.go b/cdc/processor/sinkmanager/table_sink_worker_test.go index bb89924f4b1..a389f097140 100644 --- a/cdc/processor/sinkmanager/table_sink_worker_test.go +++ b/cdc/processor/sinkmanager/table_sink_worker_test.go @@ -34,7 +34,10 @@ import ( // testEventSize is the size of a test event. // It is used to calculate the memory quota. -const testEventSize = 226 +var ( + emptyEvent = model.RowChangedEvent{} + testEventSize = emptyEvent.ApproximateBytes() +) //nolint:unparam func genPolymorphicEventWithNilRow(startTs, @@ -77,13 +80,16 @@ func genPolymorphicEvent(startTs, commitTs uint64, span tablepb.Span) *model.Pol func genRowChangedEvent(startTs, commitTs uint64, span tablepb.Span) *model.RowChangedEvent { return &model.RowChangedEvent{ - StartTs: startTs, - CommitTs: commitTs, - Table: &model.TableName{ - Schema: "table", - Table: "table", - TableID: span.TableID, - IsPartition: false, + StartTs: startTs, + CommitTs: commitTs, + PhysicalTableID: span.TableID, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "table", + Table: "table", + TableID: span.TableID, + IsPartition: false, + }, }, Columns: []*model.Column{ {Name: "a", Value: 2}, @@ -105,10 +111,10 @@ func TestTableSinkWorkerSuite(t *testing.T) { } func (suite *tableSinkWorkerSuite) SetupSuite() { - requestMemSize = testEventSize + requestMemSize = uint64(testEventSize) // For one batch size. // Advance table sink per 2 events. - maxUpdateIntervalSize = testEventSize * 2 + maxUpdateIntervalSize = uint64(testEventSize * 2) suite.testChangefeedID = model.DefaultChangeFeedID("1") suite.testSpan = spanz.TableIDToComparableSpan(1) } @@ -135,7 +141,7 @@ func (suite *tableSinkWorkerSuite) createWorker( // To avoid refund or release panics. quota := memquota.NewMemQuota(suite.testChangefeedID, memQuota, "sink") // NOTICE: Do not forget the initial memory quota in the worker first time running. - quota.ForceAcquire(testEventSize) + quota.ForceAcquire(uint64(testEventSize)) quota.AddTable(suite.testSpan) return newSinkWorker(suite.testChangefeedID, sm, quota, splitTxn), sortEngine diff --git a/cdc/processor/sinkmanager/table_sink_wrapper_test.go b/cdc/processor/sinkmanager/table_sink_wrapper_test.go index 91688e974a3..689717ecf8c 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper_test.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper_test.go @@ -219,13 +219,15 @@ func TestHandleRowChangedEventNormalEvent(t *testing.T) { CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, Row: &model.RowChangedEvent{ - CommitTs: 1, + CommitTs: 1, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "test", + }, + }, Columns: columns, PreColumns: preColumns, - Table: &model.TableName{ - Schema: "test", - Table: "test", - }, }, }, } @@ -233,7 +235,7 @@ func TestHandleRowChangedEventNormalEvent(t *testing.T) { span := spanz.TableIDToComparableSpan(1) result, size := handleRowChangedEvents(changefeedID, span, events...) require.Equal(t, 1, len(result)) - require.Equal(t, uint64(224), size) + require.Equal(t, uint64(testEventSize), size) } func TestGetUpperBoundTs(t *testing.T) { diff --git a/cdc/processor/sourcemanager/sorter/mounted_iter_test.go b/cdc/processor/sourcemanager/sorter/mounted_iter_test.go index d13b4a51e6d..0a690597b01 100644 --- a/cdc/processor/sourcemanager/sorter/mounted_iter_test.go +++ b/cdc/processor/sourcemanager/sorter/mounted_iter_test.go @@ -46,8 +46,13 @@ func TestMountedEventIter(t *testing.T) { Value: []byte("testbytes"), }, Row: &model.RowChangedEvent{ - Table: &model.TableName{Schema: "schema", Table: "table"}, - IndexColumns: [][]int{{1}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "schema", + Table: "table", + }, + IndexColumnsOffset: [][]int{{1}}, + }, }, } }, diff --git a/cdc/redo/manager_test.go b/cdc/redo/manager_test.go index 381f4e99743..52c580ee3e9 100644 --- a/cdc/redo/manager_test.go +++ b/cdc/redo/manager_test.go @@ -140,6 +140,9 @@ func TestLogManagerInProcessor(t *testing.T) { for _, span := range spans { dmlMgr.AddTable(span, startTs) } + tableInfo := &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "t"}, + } testCases := []struct { span tablepb.Span rows []*model.RowChangedEvent @@ -147,30 +150,30 @@ func TestLogManagerInProcessor(t *testing.T) { { span: spanz.TableIDToComparableSpan(53), rows: []*model.RowChangedEvent{ - {CommitTs: 120, Table: &model.TableName{TableID: 53}}, - {CommitTs: 125, Table: &model.TableName{TableID: 53}}, - {CommitTs: 130, Table: &model.TableName{TableID: 53}}, + {CommitTs: 120, PhysicalTableID: 53, TableInfo: tableInfo}, + {CommitTs: 125, PhysicalTableID: 53, TableInfo: tableInfo}, + {CommitTs: 130, PhysicalTableID: 53, TableInfo: tableInfo}, }, }, { span: spanz.TableIDToComparableSpan(55), rows: []*model.RowChangedEvent{ - {CommitTs: 130, Table: &model.TableName{TableID: 55}}, - {CommitTs: 135, Table: &model.TableName{TableID: 55}}, + {CommitTs: 130, PhysicalTableID: 55, TableInfo: tableInfo}, + {CommitTs: 135, PhysicalTableID: 55, TableInfo: tableInfo}, }, }, { span: spanz.TableIDToComparableSpan(57), rows: []*model.RowChangedEvent{ - {CommitTs: 130, Table: &model.TableName{TableID: 57}}, + {CommitTs: 130, PhysicalTableID: 57, TableInfo: tableInfo}, }, }, { span: spanz.TableIDToComparableSpan(59), rows: []*model.RowChangedEvent{ - {CommitTs: 128, Table: &model.TableName{TableID: 59}}, - {CommitTs: 130, Table: &model.TableName{TableID: 59}}, - {CommitTs: 133, Table: &model.TableName{TableID: 59}}, + {CommitTs: 128, PhysicalTableID: 59, TableInfo: tableInfo}, + {CommitTs: 130, PhysicalTableID: 59, TableInfo: tableInfo}, + {CommitTs: 133, PhysicalTableID: 59, TableInfo: tableInfo}, }, }, } @@ -286,6 +289,9 @@ func TestLogManagerError(t *testing.T) { return logMgr.Run(ctx) }) + tableInfo := &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "t"}, + } testCases := []struct { span tablepb.Span rows []writer.RedoEvent @@ -293,9 +299,9 @@ func TestLogManagerError(t *testing.T) { { span: spanz.TableIDToComparableSpan(53), rows: []writer.RedoEvent{ - &model.RowChangedEvent{CommitTs: 120, Table: &model.TableName{TableID: 53}}, - &model.RowChangedEvent{CommitTs: 125, Table: &model.TableName{TableID: 53}}, - &model.RowChangedEvent{CommitTs: 130, Table: &model.TableName{TableID: 53}}, + &model.RowChangedEvent{CommitTs: 120, PhysicalTableID: 53, TableInfo: tableInfo}, + &model.RowChangedEvent{CommitTs: 125, PhysicalTableID: 53, TableInfo: tableInfo}, + &model.RowChangedEvent{CommitTs: 130, PhysicalTableID: 53, TableInfo: tableInfo}, }, }, } @@ -361,6 +367,9 @@ func runBenchTest(b *testing.B, storage string, useFileBackend bool) { b.ResetTimer() for _, tableID := range tables { wg.Add(1) + tableInfo := &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: fmt.Sprintf("t_%d", tableID)}, + } go func(span tablepb.Span) { defer wg.Done() maxCommitTs := maxTsMap.GetV(span) @@ -371,9 +380,9 @@ func runBenchTest(b *testing.B, storage string, useFileBackend bool) { b.StopTimer() *maxCommitTs += rand.Uint64() % 10 rows = []*model.RowChangedEvent{ - {CommitTs: *maxCommitTs, Table: &model.TableName{TableID: span.TableID}}, - {CommitTs: *maxCommitTs, Table: &model.TableName{TableID: span.TableID}}, - {CommitTs: *maxCommitTs, Table: &model.TableName{TableID: span.TableID}}, + {CommitTs: *maxCommitTs, PhysicalTableID: span.TableID, TableInfo: tableInfo}, + {CommitTs: *maxCommitTs, PhysicalTableID: span.TableID, TableInfo: tableInfo}, + {CommitTs: *maxCommitTs, PhysicalTableID: span.TableID, TableInfo: tableInfo}, } b.StartTimer() diff --git a/cdc/redo/reader/reader.go b/cdc/redo/reader/reader.go index 92fe02bef81..c12959cebb3 100644 --- a/cdc/redo/reader/reader.go +++ b/cdc/redo/reader/reader.go @@ -87,7 +87,7 @@ type LogReaderConfig struct { type LogReader struct { cfg *LogReaderConfig meta *common.LogMeta - rowCh chan *model.RowChangedEvent + rowCh chan *model.RowChangedEventInRedoLog ddlCh chan *model.DDLEvent } @@ -106,7 +106,7 @@ func newLogReader(ctx context.Context, cfg *LogReaderConfig) (*LogReader, error) logReader := &LogReader{ cfg: cfg, - rowCh: make(chan *model.RowChangedEvent, defaultReaderChanSize), + rowCh: make(chan *model.RowChangedEventInRedoLog, defaultReaderChanSize), ddlCh: make(chan *model.DDLEvent, defaultReaderChanSize), } // remove logs in local dir first, if have logs left belongs to previous changefeed with the same name may have error when apply logs @@ -243,8 +243,27 @@ func (l *LogReader) ReadNextRow(ctx context.Context) (*model.RowChangedEvent, er select { case <-ctx.Done(): return nil, errors.Trace(ctx.Err()) - case row := <-l.rowCh: - return row, nil + case rowInRedoLog := <-l.rowCh: + if rowInRedoLog != nil { + row := &model.RowChangedEvent{ + StartTs: rowInRedoLog.StartTs, + CommitTs: rowInRedoLog.CommitTs, + PhysicalTableID: rowInRedoLog.Table.TableID, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: rowInRedoLog.Table.Schema, + Table: rowInRedoLog.Table.Table, + TableID: rowInRedoLog.Table.TableID, + IsPartition: rowInRedoLog.Table.IsPartition, + }, + IndexColumnsOffset: rowInRedoLog.IndexColumns, + }, + Columns: rowInRedoLog.Columns, + PreColumns: rowInRedoLog.PreColumns, + } + return row, nil + } + return nil, nil } } diff --git a/cdc/redo/reader/reader_test.go b/cdc/redo/reader/reader_test.go index 1a40f055b96..9f4b3afd6a4 100644 --- a/cdc/redo/reader/reader_test.go +++ b/cdc/redo/reader/reader_test.go @@ -51,7 +51,15 @@ func genLogFile( if logType == redo.RedoRowLogFileType { // generate unsorted logs for ts := maxCommitTs; ts >= minCommitTs; ts-- { - event := &model.RowChangedEvent{CommitTs: ts} + event := &model.RowChangedEvent{ + CommitTs: ts, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t", + }, + }, + } log := event.ToRedoLog() rawData, err := codec.MarshalRedoLog(log, nil) require.Nil(t, err) @@ -101,7 +109,7 @@ func TestReadLogs(t *testing.T) { UseExternalStorage: true, }, meta: meta, - rowCh: make(chan *model.RowChangedEvent, defaultReaderChanSize), + rowCh: make(chan *model.RowChangedEventInRedoLog, defaultReaderChanSize), ddlCh: make(chan *model.DDLEvent, defaultReaderChanSize), } eg, egCtx := errgroup.WithContext(ctx) @@ -150,7 +158,7 @@ func TestLogReaderClose(t *testing.T) { UseExternalStorage: true, }, meta: meta, - rowCh: make(chan *model.RowChangedEvent, 1), + rowCh: make(chan *model.RowChangedEventInRedoLog, 1), ddlCh: make(chan *model.DDLEvent, 1), } eg, egCtx := errgroup.WithContext(ctx) diff --git a/cdc/redo/writer/file/file_log_writer_test.go b/cdc/redo/writer/file/file_log_writer_test.go index 235af74b067..b007be30917 100644 --- a/cdc/redo/writer/file/file_log_writer_test.go +++ b/cdc/redo/writer/file/file_log_writer_test.go @@ -33,6 +33,12 @@ func TestLogWriterWriteLog(t *testing.T) { ctx context.Context rows []writer.RedoEvent } + tableInfo := &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t", + }, + } tests := []struct { name string args arg @@ -46,7 +52,7 @@ func TestLogWriterWriteLog(t *testing.T) { args: arg{ ctx: context.Background(), rows: []writer.RedoEvent{ - &model.RowChangedEvent{Table: &model.TableName{TableID: 111}, CommitTs: 1}, + &model.RowChangedEvent{CommitTs: 1, PhysicalTableID: 111, TableInfo: tableInfo}, }, }, isRunning: true, @@ -58,7 +64,7 @@ func TestLogWriterWriteLog(t *testing.T) { ctx: context.Background(), rows: []writer.RedoEvent{ nil, - &model.RowChangedEvent{Table: &model.TableName{TableID: 11}, CommitTs: 11}, + &model.RowChangedEvent{CommitTs: 1, PhysicalTableID: 11, TableInfo: tableInfo}, }, }, writerErr: errors.New("err"), diff --git a/cdc/redo/writer/memory/mem_log_writer_test.go b/cdc/redo/writer/memory/mem_log_writer_test.go index 2e21d8dbe13..b20b2eb4131 100644 --- a/cdc/redo/writer/memory/mem_log_writer_test.go +++ b/cdc/redo/writer/memory/mem_log_writer_test.go @@ -33,9 +33,21 @@ func TestWriteDDL(t *testing.T) { rows := []writer.RedoEvent{ nil, - &model.RowChangedEvent{Table: &model.TableName{TableID: 11}, CommitTs: 11}, - &model.RowChangedEvent{Table: &model.TableName{TableID: 12}, CommitTs: 15}, - &model.RowChangedEvent{Table: &model.TableName{TableID: 12}, CommitTs: 8}, + &model.RowChangedEvent{ + PhysicalTableID: 11, + CommitTs: 11, + TableInfo: &model.TableInfo{TableName: model.TableName{Schema: "test", Table: "t1"}}, + }, + &model.RowChangedEvent{ + PhysicalTableID: 12, + CommitTs: 15, + TableInfo: &model.TableInfo{TableName: model.TableName{Schema: "test", Table: "t2"}}, + }, + &model.RowChangedEvent{ + PhysicalTableID: 12, + CommitTs: 8, + TableInfo: &model.TableInfo{TableName: model.TableName{Schema: "test", Table: "t2"}}, + }, } testWriteEvents(t, rows) } diff --git a/cdc/sink/dmlsink/blackhole/black_hole_dml_sink_test.go b/cdc/sink/dmlsink/blackhole/black_hole_dml_sink_test.go index d1045f5fb81..8faeeda2fc4 100644 --- a/cdc/sink/dmlsink/blackhole/black_hole_dml_sink_test.go +++ b/cdc/sink/dmlsink/blackhole/black_hole_dml_sink_test.go @@ -29,8 +29,13 @@ func TestWriteEventsCallback(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"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", + Table: "b", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, } count := 0 diff --git a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go index 77b6c1a273a..e1dca4941af 100644 --- a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go +++ b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go @@ -248,8 +248,13 @@ func (s *DMLSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa } tbl := cloudstorage.VersionedTableName{ - TableNameWithPhysicTableID: *txn.Event.Table, - TableInfoVersion: txn.Event.TableInfoVersion, + TableNameWithPhysicTableID: model.TableName{ + Schema: txn.Event.TableInfo.GetSchemaName(), + Table: txn.Event.TableInfo.GetTableName(), + TableID: txn.Event.GetPhysicalTableID(), + IsPartition: txn.Event.TableInfo.IsPartitionTable(), + }, + TableInfoVersion: txn.Event.TableInfoVersion, } seq := atomic.AddUint64(&s.lastSeqNum, 1) diff --git a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink_test.go b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink_test.go index 3d6887006f5..3d32eac2097 100644 --- a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink_test.go +++ b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink_test.go @@ -72,7 +72,6 @@ func generateTxnEvents( txn := &dmlsink.TxnCallbackableEvent{ Event: &model.SingleTableTxn{ CommitTs: 100, - Table: &model.TableName{Schema: "test", Table: "table1"}, TableInfoVersion: 33, TableInfo: &model.TableInfo{ TableName: model.TableName{ @@ -95,7 +94,6 @@ func generateTxnEvents( for j := 0; j < batch; j++ { row := &model.RowChangedEvent{ CommitTs: 100, - Table: &model.TableName{Schema: "test", Table: "table1"}, TableInfo: &model.TableInfo{TableName: model.TableName{Schema: "test", Table: "table1"}, Version: 33}, Columns: []*model.Column{ {Name: "c1", Value: i*batch + j}, diff --git a/cdc/sink/dmlsink/cloudstorage/defragmenter_test.go b/cdc/sink/dmlsink/cloudstorage/defragmenter_test.go index 184180fbf67..2d0114863c2 100644 --- a/cdc/sink/dmlsink/cloudstorage/defragmenter_test.go +++ b/cdc/sink/dmlsink/cloudstorage/defragmenter_test.go @@ -89,10 +89,13 @@ func TestDeframenter(t *testing.T) { n := 1 + rand.Intn(1000) for j := 0; j < n; j++ { row := &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, + PhysicalTableID: 100, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "table1", + TableID: 100, + }, }, Columns: []*model.Column{ {Name: "c1", Value: j + 1}, diff --git a/cdc/sink/dmlsink/cloudstorage/dml_worker_test.go b/cdc/sink/dmlsink/cloudstorage/dml_worker_test.go index b202a20f255..7ed84a82a64 100644 --- a/cdc/sink/dmlsink/cloudstorage/dml_worker_test.go +++ b/cdc/sink/dmlsink/cloudstorage/dml_worker_test.go @@ -98,10 +98,13 @@ func TestDMLWorkerRun(t *testing.T) { TableInfo: tableInfo, Rows: []*model.RowChangedEvent{ { - Table: &model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, + PhysicalTableID: 100, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "table1", + TableID: 100, + }, }, Columns: []*model.Column{ {Name: "c1", Value: 100}, diff --git a/cdc/sink/dmlsink/cloudstorage/encoding_worker_test.go b/cdc/sink/dmlsink/cloudstorage/encoding_worker_test.go index 29fdc65106e..5d628587d40 100644 --- a/cdc/sink/dmlsink/cloudstorage/encoding_worker_test.go +++ b/cdc/sink/dmlsink/cloudstorage/encoding_worker_test.go @@ -79,6 +79,13 @@ func TestEncodeEvents(t *testing.T) { Ft: types.NewFieldType(mysql.TypeString), }, } + tableInfo := &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "table1", + TableID: 100, + }, + } err := encodingWorker.encodeEvents(eventFragment{ versionedTable: cloudstorage.VersionedTableName{ TableNameWithPhysicTableID: model.TableName{ @@ -90,20 +97,11 @@ func TestEncodeEvents(t *testing.T) { seqNumber: 1, event: &dmlsink.TxnCallbackableEvent{ Event: &model.SingleTableTxn{ - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, - }, - }, + TableInfo: tableInfo, Rows: []*model.RowChangedEvent{ { - Table: &model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, - }, + PhysicalTableID: 100, + TableInfo: tableInfo, Columns: []*model.Column{ {Name: "c1", Value: 100}, {Name: "c2", Value: "hello world"}, @@ -111,11 +109,8 @@ func TestEncodeEvents(t *testing.T) { ColInfos: colInfos, }, { - Table: &model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, - }, + PhysicalTableID: 100, + TableInfo: tableInfo, Columns: []*model.Column{ {Name: "c1", Value: 200}, {Name: "c2", Value: "你好,世界"}, @@ -158,10 +153,13 @@ func TestEncodingWorkerRun(t *testing.T) { }, Rows: []*model.RowChangedEvent{ { - Table: &model.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, + PhysicalTableID: 100, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "table1", + TableID: 100, + }, }, Columns: []*model.Column{ {Name: "c1", Value: 100}, diff --git a/cdc/sink/dmlsink/event_appender.go b/cdc/sink/dmlsink/event_appender.go index d8e0aec0ae1..d2537e9ea3b 100644 --- a/cdc/sink/dmlsink/event_appender.go +++ b/cdc/sink/dmlsink/event_appender.go @@ -113,10 +113,10 @@ func (t *TxnEventAppender) createSingleTableTxn( row *model.RowChangedEvent, ) *model.SingleTableTxn { txn := &model.SingleTableTxn{ - StartTs: row.StartTs, - CommitTs: row.CommitTs, - Table: row.Table, - TableInfo: row.TableInfo, + StartTs: row.StartTs, + CommitTs: row.CommitTs, + PhysicalTableID: row.PhysicalTableID, + TableInfo: row.TableInfo, } if row.TableInfo != nil { txn.TableInfoVersion = row.TableInfo.Version diff --git a/cdc/sink/dmlsink/event_appender_test.go b/cdc/sink/dmlsink/event_appender_test.go index 0088193a8ef..1f8922623b8 100644 --- a/cdc/sink/dmlsink/event_appender_test.go +++ b/cdc/sink/dmlsink/event_appender_test.go @@ -23,13 +23,13 @@ import ( func TestRowChangeEventAppender(t *testing.T) { t.Parallel() - tableName := &model.TableName{ - Schema: "test", - Table: "t1", - TableID: 1, - IsPartition: false, - } tableInfo := &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + TableID: 1, + IsPartition: false, + }, Version: 1, } @@ -37,17 +37,14 @@ func TestRowChangeEventAppender(t *testing.T) { var buffer []*model.RowChangedEvent rows := []*model.RowChangedEvent{ { - Table: tableName, TableInfo: tableInfo, CommitTs: 1, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 2, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 2, }, @@ -63,63 +60,55 @@ func TestRowChangeEventAppender(t *testing.T) { func TestTxnEventAppenderWithoutIgnoreStartTs(t *testing.T) { t.Parallel() - tableame := &model.TableName{ - Schema: "test", - Table: "t1", - TableID: 1, - IsPartition: false, - } tableInfo := &model.TableInfo{ Version: 1, + TableName: model.TableName{ + Schema: "test", + Table: "t1", + TableID: 1, + IsPartition: false, + }, } appender := &TxnEventAppender{} var buffer []*model.SingleTableTxn rows := []*model.RowChangedEvent{ { - Table: tableame, TableInfo: tableInfo, CommitTs: 101, StartTs: 98, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 102, StartTs: 99, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 102, StartTs: 100, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 102, StartTs: 100, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 103, StartTs: 101, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 103, StartTs: 101, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 104, StartTs: 102, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 105, StartTs: 103, @@ -127,19 +116,16 @@ func TestTxnEventAppenderWithoutIgnoreStartTs(t *testing.T) { SplitTxn: true, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 105, StartTs: 103, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 105, StartTs: 103, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 105, StartTs: 103, @@ -147,7 +133,6 @@ func TestTxnEventAppenderWithoutIgnoreStartTs(t *testing.T) { SplitTxn: true, }, { - Table: tableame, TableInfo: tableInfo, CommitTs: 105, StartTs: 103, @@ -181,14 +166,14 @@ func TestTxnEventAppenderWithoutIgnoreStartTs(t *testing.T) { // Test the case which the commitTs is not strictly increasing. rows = []*model.RowChangedEvent{ { - Table: tableame, - CommitTs: 101, - StartTs: 98, + TableInfo: tableInfo, + CommitTs: 101, + StartTs: 98, }, { - Table: tableame, - CommitTs: 100, - StartTs: 99, + TableInfo: tableInfo, + CommitTs: 100, + StartTs: 99, }, } buffer = buffer[:0] @@ -199,14 +184,14 @@ func TestTxnEventAppenderWithoutIgnoreStartTs(t *testing.T) { // Test the case which the startTs is not strictly increasing. rows = []*model.RowChangedEvent{ { - Table: tableame, - CommitTs: 101, - StartTs: 98, + TableInfo: tableInfo, + CommitTs: 101, + StartTs: 98, }, { - Table: tableame, - CommitTs: 101, - StartTs: 80, + TableInfo: tableInfo, + CommitTs: 101, + StartTs: 80, }, } buffer = buffer[:0] @@ -218,63 +203,55 @@ func TestTxnEventAppenderWithoutIgnoreStartTs(t *testing.T) { func TestTxnEventAppenderWithIgnoreStartTs(t *testing.T) { t.Parallel() - tableName := &model.TableName{ - Schema: "test", - Table: "t1", - TableID: 1, - IsPartition: false, - } tableInfo := &model.TableInfo{ Version: 1, + TableName: model.TableName{ + Schema: "test", + Table: "t1", + TableID: 1, + IsPartition: false, + }, } appender := &TxnEventAppender{IgnoreStartTs: true} var buffer []*model.SingleTableTxn rows := []*model.RowChangedEvent{ { - Table: tableName, TableInfo: tableInfo, CommitTs: 101, StartTs: 0, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 101, StartTs: 0, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 102, StartTs: 90, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 102, StartTs: 91, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 103, StartTs: 0, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 103, StartTs: 0, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 104, StartTs: 0, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 105, StartTs: 0, @@ -282,19 +259,16 @@ func TestTxnEventAppenderWithIgnoreStartTs(t *testing.T) { SplitTxn: true, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 105, StartTs: 0, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 105, StartTs: 0, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 105, StartTs: 0, @@ -302,7 +276,6 @@ func TestTxnEventAppenderWithIgnoreStartTs(t *testing.T) { SplitTxn: true, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 105, StartTs: 0, @@ -338,13 +311,11 @@ func TestTxnEventAppenderWithIgnoreStartTs(t *testing.T) { // Test the case which the commitTs is not strictly increasing. rows = []*model.RowChangedEvent{ { - Table: tableName, TableInfo: tableInfo, CommitTs: 101, StartTs: 98, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 100, StartTs: 99, @@ -358,13 +329,11 @@ func TestTxnEventAppenderWithIgnoreStartTs(t *testing.T) { // Test the case which the startTs is not strictly increasing. rows = []*model.RowChangedEvent{ { - Table: tableName, TableInfo: tableInfo, CommitTs: 101, StartTs: 98, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 101, StartTs: 80, @@ -378,13 +347,11 @@ func TestTxnEventAppenderWithIgnoreStartTs(t *testing.T) { // Test the case which the startTs all is 0. rows = []*model.RowChangedEvent{ { - Table: tableName, TableInfo: tableInfo, CommitTs: 101, StartTs: 0, }, { - Table: tableName, TableInfo: tableInfo, CommitTs: 101, StartTs: 0, diff --git a/cdc/sink/dmlsink/mq/dispatcher/event_router.go b/cdc/sink/dmlsink/mq/dispatcher/event_router.go index 23f8d3b3cf1..e7d68014ff9 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/event_router.go +++ b/cdc/sink/dmlsink/mq/dispatcher/event_router.go @@ -89,8 +89,8 @@ func NewEventRouter( // GetTopicForRowChange returns the target topic for row changes. func (s *EventRouter) GetTopicForRowChange(row *model.RowChangedEvent) string { - topicDispatcher, _ := s.matchDispatcher(row.Table.Schema, row.Table.Table) - return topicDispatcher.Substitute(row.Table.Schema, row.Table.Table) + topicDispatcher, _ := s.matchDispatcher(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName()) + return topicDispatcher.Substitute(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName()) } // GetTopicForDDL returns the target topic for DDL. @@ -119,7 +119,7 @@ func (s *EventRouter) GetPartitionForRowChange( row *model.RowChangedEvent, partitionNum int32, ) (int32, string, error) { - return s.GetPartitionDispatcher(row.Table.Schema, row.Table.Table). + return s.GetPartitionDispatcher(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName()). DispatchRowChangedEvent(row, partitionNum) } diff --git a/cdc/sink/dmlsink/mq/dispatcher/event_router_test.go b/cdc/sink/dmlsink/mq/dispatcher/event_router_test.go index 2d90d7e8527..6efc91ba456 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/event_router_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/event_router_test.go @@ -160,27 +160,37 @@ func TestGetTopicForRowChange(t *testing.T) { require.NoError(t, err) topicName := d.GetTopicForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "test_default1", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test_default1", Table: "table"}, + }, }) require.Equal(t, "test", topicName) topicName = d.GetTopicForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "test_default2", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test_default2", Table: "table"}, + }, }) require.Equal(t, "test", topicName) topicName = d.GetTopicForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "test_table", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test_table", Table: "table"}, + }, }) require.Equal(t, "hello_test_table_world", topicName) topicName = d.GetTopicForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "test_index_value", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test_index_value", Table: "table"}, + }, }) require.Equal(t, "test_index_value_world", topicName) topicName = d.GetTopicForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "a", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "table"}, + }, }) require.Equal(t, "a_table", topicName) } @@ -193,7 +203,10 @@ func TestGetPartitionForRowChange(t *testing.T) { require.NoError(t, err) p, _, err := d.GetPartitionForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "test_default1", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test_default1", Table: "table"}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "id", @@ -201,13 +214,15 @@ func TestGetPartitionForRowChange(t *testing.T) { Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, }, }, - IndexColumns: [][]int{{0}}, }, 16) require.NoError(t, err) require.Equal(t, int32(14), p) p, _, err = d.GetPartitionForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "test_default2", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test_default2", Table: "table"}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "id", @@ -215,20 +230,23 @@ func TestGetPartitionForRowChange(t *testing.T) { Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, }, }, - IndexColumns: [][]int{{0}}, }, 16) require.NoError(t, err) require.Equal(t, int32(0), p) p, _, err = d.GetPartitionForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "test_table", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test_table", Table: "table"}, + }, CommitTs: 1, }, 16) require.NoError(t, err) require.Equal(t, int32(15), p) p, _, err = d.GetPartitionForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "test_index_value", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test_index_value", Table: "table"}, + }, Columns: []*model.Column{ { Name: "a", @@ -245,7 +263,9 @@ func TestGetPartitionForRowChange(t *testing.T) { require.Equal(t, int32(1), p) p, _, err = d.GetPartitionForRowChange(&model.RowChangedEvent{ - Table: &model.TableName{Schema: "a", Table: "table"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "table"}, + }, CommitTs: 1, }, 2) require.NoError(t, err) diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/columns.go b/cdc/sink/dmlsink/mq/dispatcher/partition/columns.go index e63c1b17e03..23a7cb3b6bf 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/columns.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/columns.go @@ -48,7 +48,7 @@ func (r *ColumnsDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, defer r.lock.Unlock() r.hasher.Reset() - r.hasher.Write([]byte(row.Table.Schema), []byte(row.Table.Table)) + r.hasher.Write([]byte(row.TableInfo.GetSchemaName()), []byte(row.TableInfo.GetTableName())) dispatchCols := row.Columns if len(dispatchCols) == 0 { @@ -58,10 +58,10 @@ func (r *ColumnsDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, offsets, ok := row.TableInfo.OffsetsByNames(r.Columns) if !ok { log.Error("columns not found when dispatch event", - zap.Any("tableName", row.Table), + zap.Any("tableName", row.TableInfo.GetTableName()), zap.Strings("columns", r.Columns)) return 0, "", errors.ErrDispatcherFailed.GenWithStack( - "columns not found when dispatch event, table: %v, columns: %v", row.Table, r.Columns) + "columns not found when dispatch event, table: %v, columns: %v", row.TableInfo.GetTableName(), r.Columns) } for idx := 0; idx < len(r.Columns); idx++ { diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/columns_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/columns_test.go index d6908e4f7e0..e9cecb86a9e 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/columns_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/columns_test.go @@ -25,35 +25,36 @@ import ( func TestColumnsDispatcher(t *testing.T) { t.Parallel() - event := &model.RowChangedEvent{ - Table: &model.TableName{ + tableInfo := &model.TableInfo{ + TableName: model.TableName{ Schema: "test", Table: "t1", }, - TableInfo: &model.TableInfo{ - TableInfo: &timodel.TableInfo{ - Columns: []*timodel.ColumnInfo{ - { - Name: timodel.CIStr{ - O: "col2", - }, - Offset: 1, + TableInfo: &timodel.TableInfo{ + Columns: []*timodel.ColumnInfo{ + { + Name: timodel.CIStr{ + O: "col2", }, - { - Name: timodel.CIStr{ - O: "col1", - }, - Offset: 0, + Offset: 1, + }, + { + Name: timodel.CIStr{ + O: "col1", }, - { - Name: timodel.CIStr{ - O: "col3", - }, - Offset: 2, + Offset: 0, + }, + { + Name: timodel.CIStr{ + O: "col3", }, + Offset: 2, }, }, }, + } + event := &model.RowChangedEvent{ + TableInfo: tableInfo, Columns: []*model.Column{ { Name: "col1", diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/default_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/default_test.go index e9b246c494c..0bcfa5c9611 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/default_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/default_test.go @@ -23,11 +23,15 @@ import ( func TestDefaultDispatcher(t *testing.T) { t.Parallel() - row := &model.RowChangedEvent{ - Table: &model.TableName{ + tableInfo := &model.TableInfo{ + TableName: model.TableName{ Schema: "test", Table: "t1", }, + IndexColumnsOffset: [][]int{{0}}, + } + row := &model.RowChangedEvent{ + TableInfo: tableInfo, Columns: []*model.Column{ { Name: "id", @@ -35,7 +39,6 @@ func TestDefaultDispatcher(t *testing.T) { Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, }, }, - IndexColumns: [][]int{{0}}, } targetPartition, _, err := NewDefaultDispatcher().DispatchRowChangedEvent(row, 3) diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/index_value.go b/cdc/sink/dmlsink/mq/dispatcher/partition/index_value.go index d6e91f65470..29ec519d655 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/index_value.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/index_value.go @@ -46,7 +46,7 @@ func (r *IndexValueDispatcher) DispatchRowChangedEvent(row *model.RowChangedEven r.lock.Lock() defer r.lock.Unlock() r.hasher.Reset() - r.hasher.Write([]byte(row.Table.Schema), []byte(row.Table.Table)) + r.hasher.Write([]byte(row.TableInfo.GetSchemaName()), []byte(row.TableInfo.GetTableName())) dispatchCols := row.Columns if len(row.Columns) == 0 { @@ -67,10 +67,10 @@ func (r *IndexValueDispatcher) DispatchRowChangedEvent(row *model.RowChangedEven names, offsets, ok := row.TableInfo.IndexByName(r.IndexName) if !ok { log.Error("index not found when dispatch event", - zap.Any("tableName", row.Table), + zap.Any("tableName", row.TableInfo.GetTableName()), zap.String("indexName", r.IndexName)) return 0, "", errors.ErrDispatcherFailed.GenWithStack( - "index not found when dispatch event, table: %v, index: %s", row.Table, r.IndexName) + "index not found when dispatch event, table: %v, index: %s", row.TableInfo.GetTableName(), r.IndexName) } for idx := 0; idx < len(names); idx++ { col := dispatchCols[offsets[idx]] diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_test.go index e7dd2e9bd00..93dbfe906da 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_test.go @@ -30,9 +30,11 @@ func TestIndexValueDispatcher(t *testing.T) { expectPartition int32 }{ {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, Columns: []*model.Column{ { @@ -47,9 +49,11 @@ func TestIndexValueDispatcher(t *testing.T) { }, }, expectPartition: 2}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, Columns: []*model.Column{ { @@ -64,9 +68,11 @@ func TestIndexValueDispatcher(t *testing.T) { }, }, expectPartition: 11}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, Columns: []*model.Column{ { @@ -81,9 +87,11 @@ func TestIndexValueDispatcher(t *testing.T) { }, }, expectPartition: 2}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, Columns: []*model.Column{ { @@ -98,9 +106,11 @@ func TestIndexValueDispatcher(t *testing.T) { }, }, expectPartition: 5}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, Columns: []*model.Column{ { @@ -115,9 +125,11 @@ func TestIndexValueDispatcher(t *testing.T) { }, }, expectPartition: 5}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, Columns: []*model.Column{ { @@ -132,9 +144,11 @@ func TestIndexValueDispatcher(t *testing.T) { }, }, expectPartition: 14}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, Columns: []*model.Column{ { @@ -160,29 +174,30 @@ func TestIndexValueDispatcher(t *testing.T) { func TestIndexValueDispatcherWithIndexName(t *testing.T) { t.Parallel() - event := &model.RowChangedEvent{ - Table: &model.TableName{ + tableInfo := &model.TableInfo{ + TableName: model.TableName{ Schema: "test", Table: "t1", }, - TableInfo: &model.TableInfo{ - TableInfo: &timodel.TableInfo{ - Indices: []*timodel.IndexInfo{ - { - Name: timodel.CIStr{ - O: "index1", - }, - Columns: []*timodel.IndexColumn{ - { - Name: timodel.CIStr{ - O: "a", - }, + TableInfo: &timodel.TableInfo{ + Indices: []*timodel.IndexInfo{ + { + Name: timodel.CIStr{ + O: "index1", + }, + Columns: []*timodel.IndexColumn{ + { + Name: timodel.CIStr{ + O: "a", }, }, }, }, }, }, + } + event := &model.RowChangedEvent{ + TableInfo: tableInfo, Columns: []*model.Column{ { Name: "a", diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/table.go b/cdc/sink/dmlsink/mq/dispatcher/partition/table.go index 0e9ad15653d..387daa033a0 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/table.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/table.go @@ -41,6 +41,6 @@ func (t *TableDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, pa defer t.lock.Unlock() t.hasher.Reset() // distribute partition by table - t.hasher.Write([]byte(row.Table.Schema), []byte(row.Table.Table)) - return int32(t.hasher.Sum32() % uint32(partitionNum)), row.Table.String(), nil + t.hasher.Write([]byte(row.TableInfo.GetSchemaName()), []byte(row.TableInfo.GetTableName())) + return int32(t.hasher.Sum32() % uint32(partitionNum)), row.TableInfo.TableName.String(), nil } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/table_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/table_test.go index 55fafcf2719..d60d9ac3006 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/table_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/table_test.go @@ -28,51 +28,65 @@ func TestTableDispatcher(t *testing.T) { expectPartition int32 }{ {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, CommitTs: 1, }, expectPartition: 15}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, CommitTs: 2, }, expectPartition: 15}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, CommitTs: 3, }, expectPartition: 15}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, CommitTs: 1, }, expectPartition: 5}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, CommitTs: 2, }, expectPartition: 5}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, CommitTs: 3, }, expectPartition: 5}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t3", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t3", + }, }, CommitTs: 3, }, expectPartition: 3}, diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/ts_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/ts_test.go index 08e6e1ba9f2..17323dcf7c6 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/ts_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/ts_test.go @@ -28,44 +28,56 @@ func TestTsDispatcher(t *testing.T) { expectPartition int32 }{ {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, CommitTs: 1, }, expectPartition: 1}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, CommitTs: 2, }, expectPartition: 2}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, CommitTs: 3, }, expectPartition: 3}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, CommitTs: 1, }, expectPartition: 1}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, CommitTs: 2, }, expectPartition: 2}, {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t2", + }, }, CommitTs: 3, }, expectPartition: 3}, diff --git a/cdc/sink/dmlsink/mq/mq_dml_sink_test.go b/cdc/sink/dmlsink/mq/mq_dml_sink_test.go index 3a84c5f136a..f518723f2bc 100644 --- a/cdc/sink/dmlsink/mq/mq_dml_sink_test.go +++ b/cdc/sink/dmlsink/mq/mq_dml_sink_test.go @@ -92,7 +92,6 @@ func TestWriteEvents(t *testing.T) { tableStatus := state.TableSinkSinking row := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "aa"}}, ColInfos: colInfo, diff --git a/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector.go b/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector.go index 2e96bad32ca..0f43f3056f1 100644 --- a/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector.go +++ b/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector.go @@ -58,9 +58,9 @@ func (s *selector) Match(schema, table string) bool { // the caller's should make sure the given event match the selector first before apply it. func (s *selector) Apply(event *model.RowChangedEvent) error { // defensive check, this should not happen. - if !s.Match(event.Table.Schema, event.Table.Table) { + if !s.Match(event.TableInfo.GetSchemaName(), event.TableInfo.GetTableName()) { return errors.ErrColumnSelectorFailed.GenWithStack( - "the given event does not match the column selector, table: %v", event.Table) + "the given event does not match the column selector, table: %v", event.TableInfo.TableName) } retainedColumns := make(map[string]struct{}, len(event.Columns)) @@ -76,7 +76,7 @@ func (s *selector) Apply(event *model.RowChangedEvent) error { if !verifyIndices(event.TableInfo, retainedColumns) { return errors.ErrColumnSelectorFailed.GenWithStack( "no primary key columns or unique key columns obtained after filter out, "+ - "table: %+v", event.Table) + "table: %+v", event.TableInfo.TableName) } } @@ -92,7 +92,7 @@ func (s *selector) Apply(event *model.RowChangedEvent) error { if !verifyIndices(event.TableInfo, retainedColumns) { return errors.ErrColumnSelectorFailed.GenWithStack( "no primary key columns or unique key columns obtained after filter out, "+ - "table: %+v", event.Table) + "table: %+v", event.TableInfo.TableName) } } @@ -124,7 +124,7 @@ func New(cfg *config.ReplicaConfig) (*ColumnSelector, error) { // Apply the column selector to the given event. func (c *ColumnSelector) Apply(event *model.RowChangedEvent) error { for _, s := range c.selectors { - if s.Match(event.Table.Schema, event.Table.Table) { + if s.Match(event.TableInfo.GetSchemaName(), event.TableInfo.GetTableName()) { return s.Apply(event) } } diff --git a/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector_test.go b/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector_test.go index 30e5aa56307..5071b1d8036 100644 --- a/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector_test.go +++ b/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector_test.go @@ -36,9 +36,11 @@ func TestNewColumnSelector(t *testing.T) { require.Len(t, selectors.selectors, 0) event := &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "table1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "table1", + }, }, Columns: []*model.Column{ { @@ -127,7 +129,6 @@ func TestVerifyTables(t *testing.T) { event4Test := func() *model.RowChangedEvent { return &model.RowChangedEvent{ - Table: &model.TableName{Schema: "test", Table: "t1"}, TableInfo: tableInfo, Columns: []*model.Column{ { diff --git a/cdc/sink/dmlsink/mq/worker_test.go b/cdc/sink/dmlsink/mq/worker_test.go index f9877acd33c..4e292add60a 100644 --- a/cdc/sink/dmlsink/mq/worker_test.go +++ b/cdc/sink/dmlsink/mq/worker_test.go @@ -87,7 +87,6 @@ func TestNonBatchEncode_SendMessages(t *testing.T) { } row := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "aa"}}, ColInfos: colInfo, @@ -147,8 +146,13 @@ 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"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", + Table: "b", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, } for i := 0; i < 512; i++ { @@ -196,8 +200,13 @@ func TestBatchEncode_Group(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", + Table: "b", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -208,8 +217,13 @@ func TestBatchEncode_Group(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", + Table: "b", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -220,8 +234,13 @@ func TestBatchEncode_Group(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 3, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", + Table: "b", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -232,8 +251,13 @@ func TestBatchEncode_Group(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 2, - Table: &model.TableName{Schema: "aa", Table: "bb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "aa", + Table: "bb", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -244,8 +268,13 @@ func TestBatchEncode_Group(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 2, - Table: &model.TableName{Schema: "aaa", Table: "bbb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "aaa", + Table: "bbb", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, Callback: func() {}, SinkState: &tableStatus, @@ -295,7 +324,6 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, ColInfos: colInfo, @@ -309,8 +337,13 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", + Table: "b", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, Callback: func() {}, SinkState: &replicatingStatus, @@ -321,8 +354,13 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 3, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", + Table: "b", + }, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, }, Callback: func() {}, SinkState: &stoppedStatus, @@ -375,7 +413,6 @@ func TestBatchEncode_SendMessages(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "aa"}}, ColInfos: colInfo, @@ -389,7 +426,6 @@ func TestBatchEncode_SendMessages(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 2, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, ColInfos: colInfo, @@ -403,7 +439,6 @@ func TestBatchEncode_SendMessages(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 3, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "cc"}}, ColInfos: colInfo, @@ -417,7 +452,6 @@ func TestBatchEncode_SendMessages(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 2, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, ColInfos: colInfo, @@ -431,7 +465,6 @@ func TestBatchEncode_SendMessages(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 2, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, ColInfos: colInfo, @@ -445,7 +478,6 @@ func TestBatchEncode_SendMessages(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 3, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "bb"}}, ColInfos: colInfo, @@ -534,7 +566,6 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "aa"}}, ColInfos: colInfo, @@ -548,7 +579,6 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 2, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, ColInfos: colInfo, @@ -562,7 +592,6 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ CommitTs: 3, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "cc"}}, ColInfos: colInfo, diff --git a/cdc/sink/dmlsink/txn/event.go b/cdc/sink/dmlsink/txn/event.go index 00805917ef0..299d96f44d5 100644 --- a/cdc/sink/dmlsink/txn/event.go +++ b/cdc/sink/dmlsink/txn/event.go @@ -107,8 +107,8 @@ func genTxnKeys(txn *model.SingleTableTxn) []uint64 { func genRowKeys(row *model.RowChangedEvent) [][]byte { var keys [][]byte if len(row.Columns) != 0 { - for iIdx, idxCol := range row.IndexColumns { - key := genKeyList(row.Columns, iIdx, idxCol, row.Table.TableID) + for iIdx, idxCol := range row.TableInfo.IndexColumnsOffset { + key := genKeyList(row.Columns, iIdx, idxCol, row.PhysicalTableID) if len(key) == 0 { continue } @@ -116,8 +116,8 @@ func genRowKeys(row *model.RowChangedEvent) [][]byte { } } if len(row.PreColumns) != 0 { - for iIdx, idxCol := range row.IndexColumns { - key := genKeyList(row.PreColumns, iIdx, idxCol, row.Table.TableID) + for iIdx, idxCol := range row.TableInfo.IndexColumnsOffset { + key := genKeyList(row.PreColumns, iIdx, idxCol, row.PhysicalTableID) if len(key) == 0 { continue } @@ -127,9 +127,9 @@ func genRowKeys(row *model.RowChangedEvent) [][]byte { if len(keys) == 0 { // use table ID as key if no key generated (no PK/UK), // no concurrence for rows in the same table. - log.Debug("Use table id as the key", zap.Int64("tableID", row.Table.TableID)) + log.Debug("Use table id as the key", zap.Int64("tableID", row.PhysicalTableID)) tableKey := make([]byte, 8) - binary.BigEndian.PutUint64(tableKey, uint64(row.Table.TableID)) + binary.BigEndian.PutUint64(tableKey, uint64(row.PhysicalTableID)) keys = [][]byte{tableKey} } return keys diff --git a/cdc/sink/dmlsink/txn/event_test.go b/cdc/sink/dmlsink/txn/event_test.go index 45999132d59..e98f3438140 100644 --- a/cdc/sink/dmlsink/txn/event_test.go +++ b/cdc/sink/dmlsink/txn/event_test.go @@ -59,9 +59,13 @@ func TestGenKeys(t *testing.T) { txn: &model.SingleTableTxn{ Rows: []*model.RowChangedEvent{ { - StartTs: 418658114257813514, - CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + PhysicalTableID: 47, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{ nil, { @@ -77,11 +81,15 @@ func TestGenKeys(t *testing.T) { Value: 1, }, }, - IndexColumns: [][]int{{1, 2}}, }, { - StartTs: 418658114257813514, - CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + PhysicalTableID: 47, + + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{ nil, { @@ -97,7 +105,6 @@ func TestGenKeys(t *testing.T) { Value: 21, }, }, - IndexColumns: [][]int{{1, 2}}, }, }, }, @@ -106,9 +113,14 @@ func TestGenKeys(t *testing.T) { txn: &model.SingleTableTxn{ Rows: []*model.RowChangedEvent{ { - StartTs: 418658114257813514, - CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + PhysicalTableID: 47, + + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1}, {2}}, + }, PreColumns: []*model.Column{ nil, { @@ -124,11 +136,14 @@ func TestGenKeys(t *testing.T) { Value: 1, }, }, - IndexColumns: [][]int{{1}, {2}}, }, { - StartTs: 418658114257813514, - CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + PhysicalTableID: 47, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1}, {2}}, + }, PreColumns: []*model.Column{ nil, { @@ -144,7 +159,6 @@ func TestGenKeys(t *testing.T) { Value: 21, }, }, - IndexColumns: [][]int{{1}, {2}}, }, }, }, @@ -153,9 +167,13 @@ func TestGenKeys(t *testing.T) { txn: &model.SingleTableTxn{ Rows: []*model.RowChangedEvent{ { - StartTs: 418658114257813514, - CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + PhysicalTableID: 47, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1}, {2}}, + }, PreColumns: []*model.Column{ nil, { @@ -171,11 +189,14 @@ func TestGenKeys(t *testing.T) { Value: nil, }, }, - IndexColumns: [][]int{{1}, {2}}, }, { - StartTs: 418658114257813514, - CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + PhysicalTableID: 47, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1}, {2}}, + }, PreColumns: []*model.Column{ nil, { @@ -191,7 +212,6 @@ func TestGenKeys(t *testing.T) { Value: 21, }, }, - IndexColumns: [][]int{{1}, {2}}, }, }, }, diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index fbca3a6fee4..b093af2bc4c 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -314,7 +314,7 @@ func convert2RowChanges( switch changeType { case sqlmodel.RowChangeInsert: res = sqlmodel.NewRowChange( - row.Table, + &row.TableInfo.TableName, nil, nil, postValues, @@ -322,7 +322,7 @@ func convert2RowChanges( nil, nil) case sqlmodel.RowChangeUpdate: res = sqlmodel.NewRowChange( - row.Table, + &row.TableInfo.TableName, nil, preValues, postValues, @@ -330,7 +330,7 @@ func convert2RowChanges( nil, nil) case sqlmodel.RowChangeDelete: res = sqlmodel.NewRowChange( - row.Table, + &row.TableInfo.TableName, nil, preValues, nil, @@ -570,8 +570,11 @@ func (s *mysqlBackend) prepareDMLs() *preparedDMLs { } // only use batch dml when the table has a handle key if hasHandleKey(tableColumns) { - // TODO(dongmen): find a better way to get table info. - tableInfo := model.BuildTiDBTableInfo(firstRow.Table.Table, tableColumns, firstRow.IndexColumns) + // TODO: will use firstRow.TableInfo.TableInfo directly after we build a more complete TableInfo in later pr + tableInfo := model.BuildTiDBTableInfo( + firstRow.TableInfo.GetTableName(), + tableColumns, + firstRow.TableInfo.IndexColumnsOffset) sql, value := s.batchSingleTxnDmls(event, tableInfo, translateToInsert) sqls = append(sqls, sql...) values = append(values, value...) @@ -586,7 +589,7 @@ func (s *mysqlBackend) prepareDMLs() *preparedDMLs { } } - quoteTable := firstRow.Table.QuoteString() + quoteTable := firstRow.TableInfo.TableName.QuoteString() for _, row := range event.Event.Rows { var query string var args []interface{} diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index 3edf2b891b0..179659a4a6d 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -119,7 +119,10 @@ func TestPrepareDML(t *testing.T) { { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{ nil, { @@ -135,7 +138,6 @@ func TestPrepareDML(t *testing.T) { Value: 1, }, }, - IndexColumns: [][]int{{1, 2}}, }, }, expected: &preparedDMLs{ @@ -152,7 +154,10 @@ func TestPrepareDML(t *testing.T) { { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{ nil, { @@ -168,7 +173,6 @@ func TestPrepareDML(t *testing.T) { Value: 2, }, }, - IndexColumns: [][]int{{1, 2}}, }, }, expected: &preparedDMLs{ @@ -328,9 +332,13 @@ func TestNewMySQLBackendExecDML(t *testing.T) { rows := []*model.RowChangedEvent{ { - StartTs: 1, - CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + StartTs: 1, + CommitTs: 2, + PhysicalTableID: 1, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1"}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -345,12 +353,15 @@ func TestNewMySQLBackendExecDML(t *testing.T) { Value: "test", }, }, - IndexColumns: [][]int{{0}}, }, { - StartTs: 5, - CommitTs: 6, - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + StartTs: 5, + CommitTs: 6, + PhysicalTableID: 1, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1"}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -365,7 +376,6 @@ func TestNewMySQLBackendExecDML(t *testing.T) { Value: "test", }, }, - IndexColumns: [][]int{{0}}, }, } @@ -391,7 +401,11 @@ func TestNewMySQLBackendExecDML(t *testing.T) { func TestExecDMLRollbackErrDatabaseNotExists(t *testing.T) { rows := []*model.RowChangedEvent{ { - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + PhysicalTableID: 1, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1"}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -400,10 +414,13 @@ func TestExecDMLRollbackErrDatabaseNotExists(t *testing.T) { Value: 1, }, }, - IndexColumns: [][]int{{0}}, }, { - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + PhysicalTableID: 1, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1"}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -412,7 +429,6 @@ func TestExecDMLRollbackErrDatabaseNotExists(t *testing.T) { Value: 2, }, }, - IndexColumns: [][]int{{0}}, }, } @@ -464,7 +480,10 @@ func TestExecDMLRollbackErrDatabaseNotExists(t *testing.T) { func TestExecDMLRollbackErrTableNotExists(t *testing.T) { rows := []*model.RowChangedEvent{ { - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -473,10 +492,12 @@ func TestExecDMLRollbackErrTableNotExists(t *testing.T) { Value: 1, }, }, - IndexColumns: [][]int{{0}}, }, { - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -485,7 +506,6 @@ func TestExecDMLRollbackErrTableNotExists(t *testing.T) { Value: 2, }, }, - IndexColumns: [][]int{{0}}, }, } @@ -537,7 +557,10 @@ func TestExecDMLRollbackErrTableNotExists(t *testing.T) { func TestExecDMLRollbackErrRetryable(t *testing.T) { rows := []*model.RowChangedEvent{ { - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -546,10 +569,12 @@ func TestExecDMLRollbackErrRetryable(t *testing.T) { Value: 1, }, }, - IndexColumns: [][]int{{0}}, }, { - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -558,7 +583,6 @@ func TestExecDMLRollbackErrRetryable(t *testing.T) { Value: 2, }, }, - IndexColumns: [][]int{{0}}, }, } @@ -617,7 +641,10 @@ func TestMysqlSinkNotRetryErrDupEntry(t *testing.T) { StartTs: 2, CommitTs: 3, ReplicatingTs: 1, - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -626,7 +653,6 @@ func TestMysqlSinkNotRetryErrDupEntry(t *testing.T) { Value: 1, }, }, - IndexColumns: [][]int{{0}}, }, } @@ -842,7 +868,10 @@ func TestMySQLSinkExecDMLError(t *testing.T) { { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -857,12 +886,14 @@ func TestMySQLSinkExecDMLError(t *testing.T) { Value: "test", }, }, - IndexColumns: [][]int{{0}}, }, { StartTs: 2, CommitTs: 3, - Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -877,7 +908,6 @@ func TestMySQLSinkExecDMLError(t *testing.T) { Value: "test", }, }, - IndexColumns: [][]int{{0}}, }, } @@ -912,7 +942,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813514, CommitTs: 418658114257813515, ReplicatingTs: 418658114257813513, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -942,7 +974,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813514, CommitTs: 418658114257813515, ReplicatingTs: 418658114257813513, - Table: &model.TableName{Schema: "common_1", Table: "pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "pk"}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -970,7 +1004,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813516, CommitTs: 418658114257813517, ReplicatingTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1012,7 +1048,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813516, CommitTs: 418658114257813517, ReplicatingTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "pk"}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1052,7 +1090,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813516, CommitTs: 418658114257813517, ReplicatingTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "pk"}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1069,7 +1109,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813516, CommitTs: 418658114257813517, ReplicatingTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "pk"}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1100,7 +1142,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813516, CommitTs: 418658114257813517, ReplicatingTs: 418658114257813518, - Table: &model.TableName{Schema: "common_1", Table: "pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "pk"}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1130,7 +1174,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813516, CommitTs: 418658114257813517, ReplicatingTs: 418658114257813518, - Table: &model.TableName{Schema: "common_1", Table: "pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "pk"}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1147,7 +1193,9 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { StartTs: 418658114257813516, CommitTs: 418658114257813517, ReplicatingTs: 418658114257813518, - Table: &model.TableName{Schema: "common_1", Table: "pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "pk"}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1211,7 +1259,10 @@ func TestPrepareBatchDMLs(t *testing.T) { { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1226,13 +1277,15 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("你好"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1247,7 +1300,6 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("世界"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, }, @@ -1265,7 +1317,10 @@ func TestPrepareBatchDMLs(t *testing.T) { { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1280,13 +1335,15 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: "你好", }}, - IndexColumns: [][]int{{1, 1}}, ApproximateDataSize: 10, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1301,7 +1358,6 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.HandleKeyFlag, Value: "世界", }}, - IndexColumns: [][]int{{2, 2}}, ApproximateDataSize: 10, }, }, @@ -1320,7 +1376,10 @@ func TestPrepareBatchDMLs(t *testing.T) { { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1349,13 +1408,15 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("测试"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 12, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1384,7 +1445,6 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("北京"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 12, }, }, @@ -1409,7 +1469,10 @@ func TestPrepareBatchDMLs(t *testing.T) { { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1424,13 +1487,15 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("你好"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1445,13 +1510,15 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("世界"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1466,13 +1533,15 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: "你好", }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1501,13 +1570,15 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("测试"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1536,7 +1607,6 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("北京"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, }, @@ -1569,7 +1639,10 @@ func TestPrepareBatchDMLs(t *testing.T) { { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1600,13 +1673,15 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("测试"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1637,7 +1712,6 @@ func TestPrepareBatchDMLs(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: []byte("北京"), }}, - IndexColumns: [][]int{{1, 2}}, ApproximateDataSize: 10, }, }, @@ -1687,7 +1761,10 @@ func TestGroupRowsByType(t *testing.T) { { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1701,12 +1778,14 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 1, }}, - IndexColumns: [][]int{{1, 2}}, }, { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1720,12 +1799,14 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 2, }}, - IndexColumns: [][]int{{1, 2}}, }, { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1739,12 +1820,14 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 2, }}, - IndexColumns: [][]int{{1, 2}}, }, { StartTs: 418658114257813514, CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, PreColumns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1758,7 +1841,6 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 2, }}, - IndexColumns: [][]int{{1, 2}}, }, }, maxTxnRow: 2, @@ -1769,7 +1851,10 @@ func TestGroupRowsByType(t *testing.T) { { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1783,12 +1868,14 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 1, }}, - IndexColumns: [][]int{{1, 2}}, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1802,12 +1889,14 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 2, }}, - IndexColumns: [][]int{{1, 2}}, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1821,12 +1910,14 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 2, }}, - IndexColumns: [][]int{{1, 2}}, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1840,13 +1931,15 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 2, }}, - IndexColumns: [][]int{{1, 2}}, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1860,13 +1953,15 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 2, }}, - IndexColumns: [][]int{{1, 2}}, }, { StartTs: 418658114257813516, CommitTs: 418658114257813517, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + IndexColumnsOffset: [][]int{{1, 2}}, + }, Columns: []*model.Column{nil, { Name: "a1", Type: mysql.TypeLong, @@ -1880,7 +1975,6 @@ func TestGroupRowsByType(t *testing.T) { model.HandleKeyFlag | model.UniqueKeyFlag, Value: 2, }}, - IndexColumns: [][]int{{1, 2}}, }, }, maxTxnRow: 4, @@ -1895,7 +1989,7 @@ func TestGroupRowsByType(t *testing.T) { if len(colums) == 0 { colums = tc.input[0].PreColumns } - tableInfo := model.BuildTiDBTableInfo("t", colums, tc.input[0].IndexColumns) + tableInfo := model.BuildTiDBTableInfo("t", colums, tc.input[0].TableInfo.IndexColumnsOffset) ms.cfg.MaxTxnRow = tc.maxTxnRow inserts, updates, deletes := ms.groupRowsByType(event, tableInfo, false) for _, rows := range inserts { diff --git a/cdc/sink/dmlsink/txn/txn_dml_sink_test.go b/cdc/sink/dmlsink/txn/txn_dml_sink_test.go index 68269123231..c5a20f56e4e 100644 --- a/cdc/sink/dmlsink/txn/txn_dml_sink_test.go +++ b/cdc/sink/dmlsink/txn/txn_dml_sink_test.go @@ -79,7 +79,9 @@ func TestTxnSinkNolocking(t *testing.T) { Event: &model.SingleTableTxn{ Rows: []*model.RowChangedEvent{ { - Table: &model.TableName{Schema: "test", Table: "t1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "t1"}, + }, Columns: []*model.Column{ {Name: "a", Value: 1}, {Name: "b", Value: 2}, diff --git a/cdc/sink/tablesink/table_sink_impl_test.go b/cdc/sink/tablesink/table_sink_impl_test.go index 34e334f334b..e925ea05c06 100644 --- a/cdc/sink/tablesink/table_sink_impl_test.go +++ b/cdc/sink/tablesink/table_sink_impl_test.go @@ -84,77 +84,79 @@ func (m *mockEventSink) acknowledge(commitTs uint64) []*dmlsink.TxnCallbackableE } func getTestRows() []*model.RowChangedEvent { - tableInfo := &model.TableName{ - Schema: "test", - Table: "t1", - TableID: 1, - IsPartition: false, + tableInfo := &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + TableID: 1, + IsPartition: false, + }, } return []*model.RowChangedEvent{ { - Table: tableInfo, - CommitTs: 101, - StartTs: 98, + TableInfo: tableInfo, + CommitTs: 101, + StartTs: 98, }, { - Table: tableInfo, - CommitTs: 102, - StartTs: 99, + TableInfo: tableInfo, + CommitTs: 102, + StartTs: 99, }, { - Table: tableInfo, - CommitTs: 102, - StartTs: 100, + TableInfo: tableInfo, + CommitTs: 102, + StartTs: 100, }, { - Table: tableInfo, - CommitTs: 102, - StartTs: 100, + TableInfo: tableInfo, + CommitTs: 102, + StartTs: 100, }, { - Table: tableInfo, - CommitTs: 103, - StartTs: 101, + TableInfo: tableInfo, + CommitTs: 103, + StartTs: 101, }, { - Table: tableInfo, - CommitTs: 103, - StartTs: 101, + TableInfo: tableInfo, + CommitTs: 103, + StartTs: 101, }, { - Table: tableInfo, - CommitTs: 104, - StartTs: 102, + TableInfo: tableInfo, + CommitTs: 104, + StartTs: 102, }, { - Table: tableInfo, - CommitTs: 105, - StartTs: 103, + TableInfo: tableInfo, + CommitTs: 105, + StartTs: 103, // Batch1 SplitTxn: true, }, { - Table: tableInfo, - CommitTs: 105, - StartTs: 103, + TableInfo: tableInfo, + CommitTs: 105, + StartTs: 103, }, { - Table: tableInfo, - CommitTs: 105, - StartTs: 103, + TableInfo: tableInfo, + CommitTs: 105, + StartTs: 103, }, { - Table: tableInfo, - CommitTs: 105, - StartTs: 103, + TableInfo: tableInfo, + CommitTs: 105, + StartTs: 103, // Batch2 SplitTxn: true, }, { - Table: tableInfo, - CommitTs: 105, - StartTs: 103, + TableInfo: tableInfo, + CommitTs: 105, + StartTs: 103, }, } } diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index 493c2d74af7..9b7057fe7f7 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -685,12 +685,12 @@ func (c *Consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim saram continue } var partitionID int64 - if row.Table.IsPartition { - partitionID = row.Table.TableID + if row.TableInfo.TableName.IsPartition { + partitionID = row.PhysicalTableID } tableID := c.fakeTableIDGenerator. - generateFakeTableID(row.Table.Schema, row.Table.Table, partitionID) - row.Table.TableID = tableID + generateFakeTableID(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName(), partitionID) + row.TableInfo.TableName.TableID = tableID group, ok := eventGroups[tableID] if !ok { diff --git a/cmd/pulsar-consumer/main.go b/cmd/pulsar-consumer/main.go index 475507d301f..c78ce80409f 100644 --- a/cmd/pulsar-consumer/main.go +++ b/cmd/pulsar-consumer/main.go @@ -463,13 +463,13 @@ func (c *Consumer) HandleMsg(msg pulsar.Message) error { continue } var partitionID int64 - if row.Table.IsPartition { - partitionID = row.Table.TableID + if row.TableInfo.TableName.IsPartition { + partitionID = row.PhysicalTableID } // use schema, table and tableID to identify a table tableID := c.fakeTableIDGenerator. - generateFakeTableID(row.Table.Schema, row.Table.Table, partitionID) - row.Table.TableID = tableID + generateFakeTableID(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName(), partitionID) + row.TableInfo.TableName.TableID = tableID group, ok := c.eventGroups[tableID] if !ok { diff --git a/cmd/storage-consumer/main.go b/cmd/storage-consumer/main.go index d9d20f41a44..6e467d34812 100644 --- a/cmd/storage-consumer/main.go +++ b/cmd/storage-consumer/main.go @@ -370,7 +370,7 @@ func (c *consumer) emitDMLEvents( ) continue } - row.Table.TableID = tableID + row.PhysicalTableID = tableID c.tableSinkMap[tableID].AppendRowChangedEvents(row) filteredCnt++ } diff --git a/pkg/applier/redo.go b/pkg/applier/redo.go index 708e5e44fb0..396f5b7bcf6 100644 --- a/pkg/applier/redo.go +++ b/pkg/applier/redo.go @@ -305,7 +305,7 @@ func (ra *RedoApplier) applyRow( } ra.pendingQuota -= rowSize - tableID := row.Table.TableID + tableID := row.PhysicalTableID if _, ok := ra.tableSinks[tableID]; !ok { tableSink := ra.sinkFactory.CreateTableSink( model.DefaultChangeFeedID(applierChangefeed), diff --git a/pkg/applier/redo_test.go b/pkg/applier/redo_test.go index e78cad66010..0f0de14fbfb 100644 --- a/pkg/applier/redo_test.go +++ b/pkg/applier/redo_test.go @@ -129,7 +129,10 @@ func TestApply(t *testing.T) { { StartTs: 1100, CommitTs: 1200, - Table: &model.TableName{Schema: "test", Table: "t1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "t1"}, + IndexColumnsOffset: [][]int{{0}}, + }, Columns: []*model.Column{ { Name: "a", @@ -141,12 +144,14 @@ func TestApply(t *testing.T) { Flag: 0, }, }, - IndexColumns: [][]int{{0}}, }, { StartTs: 1200, CommitTs: resolvedTs, - Table: &model.TableName{Schema: "test", Table: "t1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "t1"}, + IndexColumnsOffset: [][]int{{0}}, + }, PreColumns: []*model.Column{ { Name: "a", @@ -169,7 +174,6 @@ func TestApply(t *testing.T) { Flag: 0, }, }, - IndexColumns: [][]int{{0}}, }, } for _, dml := range dmls { diff --git a/pkg/filter/expr_filter.go b/pkg/filter/expr_filter.go index 859f750e20e..61835c6d692 100644 --- a/pkg/filter/expr_filter.go +++ b/pkg/filter/expr_filter.go @@ -423,7 +423,7 @@ func (f *dmlExprFilter) shouldSkipDML( if ti == nil || row == nil || rawRow.IsEmpty() { return false, nil } - rules := f.getRules(row.Table.Schema, row.Table.Table) + rules := f.getRules(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName()) for _, rule := range rules { ignore, err := rule.shouldSkipDML(row, rawRow, ti) if err != nil { diff --git a/pkg/filter/expr_filter_bench_test.go b/pkg/filter/expr_filter_bench_test.go index e5d3b5c14a8..4cb1d051ef6 100644 --- a/pkg/filter/expr_filter_bench_test.go +++ b/pkg/filter/expr_filter_bench_test.go @@ -130,9 +130,11 @@ func BenchmarkSkipDML(b *testing.B) { preRowDatums, err := utils.AdjustBinaryProtocolForDatum(sessCtx, c.preRow, tableInfo.Columns) require.Nil(t, err) row := &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: c.schema, - Table: c.table, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: c.schema, + Table: c.table, + }, }, Columns: c.columns, PreColumns: c.preColumns, diff --git a/pkg/filter/expr_filter_test.go b/pkg/filter/expr_filter_test.go index f86e2886265..95a3e69aad4 100644 --- a/pkg/filter/expr_filter_test.go +++ b/pkg/filter/expr_filter_test.go @@ -330,9 +330,11 @@ func TestShouldSkipDMLBasic(t *testing.T) { preRowDatums, err := utils.AdjustBinaryProtocolForDatum(sessCtx, c.preRow, tableInfo.Columns) require.Nil(t, err) row := &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: c.schema, - Table: c.table, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: c.schema, + Table: c.table, + }, }, Columns: c.columns, PreColumns: c.preColumns, @@ -447,9 +449,11 @@ func TestShouldSkipDMLError(t *testing.T) { preRowDatums, err := utils.AdjustBinaryProtocolForDatum(sessCtx, c.preRow, tableInfo.Columns) require.Nil(t, err) row := &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: c.schema, - Table: c.table, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: c.schema, + Table: c.table, + }, }, Columns: c.columns, PreColumns: c.preColumns, @@ -643,9 +647,11 @@ func TestShouldSkipDMLTableUpdated(t *testing.T) { preRowDatums, err := utils.AdjustBinaryProtocolForDatum(sessCtx, c.preRow, tableInfo.Columns) require.Nil(t, err) row := &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: c.schema, - Table: c.table, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: c.schema, + Table: c.table, + }, }, Columns: c.columns, PreColumns: c.preColumns, diff --git a/pkg/filter/filter.go b/pkg/filter/filter.go index b2230d17911..f1e464f7b5a 100644 --- a/pkg/filter/filter.go +++ b/pkg/filter/filter.go @@ -136,7 +136,7 @@ func (f *filter) ShouldIgnoreDMLEvent( return true, nil } - if f.ShouldIgnoreTable(dml.Table.Schema, dml.Table.Table) { + if f.ShouldIgnoreTable(dml.TableInfo.GetSchemaName(), dml.TableInfo.GetTableName()) { return true, nil } diff --git a/pkg/filter/filter_test.go b/pkg/filter/filter_test.go index dfb0f99075b..97789821b94 100644 --- a/pkg/filter/filter_test.go +++ b/pkg/filter/filter_test.go @@ -142,7 +142,12 @@ func TestShouldIgnoreDMLEvent(t *testing.T) { require.Nil(t, err) for _, tc := range ftc.cases { dml := &model.RowChangedEvent{ - Table: &model.TableName{Table: tc.table, Schema: tc.schema}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: tc.schema, + Table: tc.table, + }, + }, StartTs: tc.ts, } ignoreDML, err := filter.ShouldIgnoreDMLEvent(dml, model.RowChangedDatums{}, nil) diff --git a/pkg/filter/sql_event_filter.go b/pkg/filter/sql_event_filter.go index 01147c7c06c..b18618eb79f 100644 --- a/pkg/filter/sql_event_filter.go +++ b/pkg/filter/sql_event_filter.go @@ -204,7 +204,7 @@ func (f *sqlEventFilter) shouldSkipDML(event *model.RowChangedEvent) (bool, erro log.Warn("unknown row changed event type") return false, nil } - rules := f.getRules(event.Table.Schema, event.Table.Table) + rules := f.getRules(event.TableInfo.GetSchemaName(), event.TableInfo.GetTableName()) for _, rule := range rules { action, err := rule.bf.Filter(binlogFilterSchemaPlaceholder, binlogFilterTablePlaceholder, et, dmlQuery) if err != nil { diff --git a/pkg/filter/sql_event_filter_test.go b/pkg/filter/sql_event_filter_test.go index 9b27fa2b9b2..27cba219ec0 100644 --- a/pkg/filter/sql_event_filter_test.go +++ b/pkg/filter/sql_event_filter_test.go @@ -301,9 +301,11 @@ func TestShouldSkipDML(t *testing.T) { require.NoError(t, err) for _, c := range tc.cases { event := &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: c.schema, - Table: c.table, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: c.schema, + Table: c.table, + }, }, } if c.columns != "" { diff --git a/pkg/sink/codec/avro/avro.go b/pkg/sink/codec/avro/avro.go index 3372f49a941..48e724b2bbc 100644 --- a/pkg/sink/codec/avro/avro.go +++ b/pkg/sink/codec/avro/avro.go @@ -86,7 +86,7 @@ func (a *BatchEncoder) encodeKey(ctx context.Context, topic string, e *model.Row columns: cols, colInfos: colInfos, } - avroCodec, header, err := a.getKeySchemaCodec(ctx, topic, e.Table, e.TableInfo.Version, keyColumns) + avroCodec, header, err := a.getKeySchemaCodec(ctx, topic, &e.TableInfo.TableName, e.TableInfo.Version, keyColumns) if err != nil { return nil, errors.Trace(err) } @@ -171,7 +171,7 @@ func (a *BatchEncoder) encodeValue(ctx context.Context, topic string, e *model.R return nil, nil } - avroCodec, header, err := a.getValueSchemaCodec(ctx, topic, e.Table, e.TableInfo.Version, input) + avroCodec, header, err := a.getValueSchemaCodec(ctx, topic, &e.TableInfo.TableName, e.TableInfo.Version, input) if err != nil { return nil, errors.Trace(err) } @@ -230,8 +230,8 @@ func (a *BatchEncoder) AppendRowChangedEvent( value, e.CommitTs, model.MessageTypeRow, - &e.Table.Schema, - &e.Table.Table, + e.TableInfo.GetSchemaNamePtr(), + e.TableInfo.GetTableNamePtr(), ) message.Callback = callback message.IncRowsCount() @@ -240,7 +240,7 @@ func (a *BatchEncoder) AppendRowChangedEvent( log.Warn("Single message is too large for avro", zap.Int("maxMessageBytes", a.config.MaxMessageBytes), zap.Int("length", message.Length()), - zap.Any("table", e.Table)) + zap.Any("table", e.TableInfo.TableName)) return cerror.ErrMessageTooLarge.GenWithStackByArgs(message.Length()) } diff --git a/pkg/sink/codec/avro/avro_test.go b/pkg/sink/codec/avro/avro_test.go index e49f98e6457..435a0ae372f 100644 --- a/pkg/sink/codec/avro/avro_test.go +++ b/pkg/sink/codec/avro/avro_test.go @@ -639,10 +639,6 @@ func newLargeEvent() *model.RowChangedEvent { return &model.RowChangedEvent{ CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "testdb", - Table: "avroencode", - }, TableInfo: &model.TableInfo{ TableName: model.TableName{ Schema: "testdb", @@ -676,7 +672,7 @@ func TestRowToAvroSchemaEnableChecksum(t *testing.T) { encoder := NewAvroEncoder(model.DefaultNamespace, nil, codecConfig) - schema, err := encoder.(*BatchEncoder).value2AvroSchema(event.Table, input) + schema, err := encoder.(*BatchEncoder).value2AvroSchema(&event.TableInfo.TableName, input) require.NoError(t, err) require.Equal(t, expectedSchemaWithExtensionEnableChecksum, indentJSON(schema)) _, err = goavro.NewCodec(schema) @@ -697,7 +693,7 @@ func TestRowToAvroSchema(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolAvro) encoder := NewAvroEncoder(model.DefaultNamespace, nil, codecConfig) - schema, err := encoder.(*BatchEncoder).value2AvroSchema(event.Table, input) + schema, err := encoder.(*BatchEncoder).value2AvroSchema(&event.TableInfo.TableName, input) require.NoError(t, err) require.Equal(t, expectedSchemaWithoutExtension, indentJSON(schema)) _, err = goavro.NewCodec(schema) @@ -706,7 +702,7 @@ func TestRowToAvroSchema(t *testing.T) { codecConfig.EnableTiDBExtension = true encoder = NewAvroEncoder(model.DefaultNamespace, nil, codecConfig) - schema, err = encoder.(*BatchEncoder).value2AvroSchema(event.Table, input) + schema, err = encoder.(*BatchEncoder).value2AvroSchema(&event.TableInfo.TableName, input) require.NoError(t, err) require.Equal(t, expectedSchemaWithExtension, indentJSON(schema)) _, err = goavro.NewCodec(schema) @@ -945,7 +941,6 @@ func TestArvoAppendRowChangedEventWithCallback(t *testing.T) { row := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, TableInfo: &model.TableInfo{TableName: model.TableName{Schema: "a", Table: "b"}}, Columns: []*model.Column{{ Name: "col1", diff --git a/pkg/sink/codec/avro/decoder.go b/pkg/sink/codec/avro/decoder.go index 20e73f95173..0b5ab457ce4 100644 --- a/pkg/sink/codec/avro/decoder.go +++ b/pkg/sink/codec/avro/decoder.go @@ -244,9 +244,11 @@ func assembleEvent(keyMap, valueMap, schema map[string]interface{}, isDelete boo event := new(model.RowChangedEvent) event.CommitTs = uint64(commitTs) - event.Table = &model.TableName{ - Schema: schemaName, - Table: tableName, + event.TableInfo = &model.TableInfo{ + TableName: model.TableName{ + Schema: schemaName, + Table: tableName, + }, } if isDelete { diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go index 0adec2d2f8a..b2a371c777e 100644 --- a/pkg/sink/codec/bootstraper.go +++ b/pkg/sink/codec/bootstraper.go @@ -107,7 +107,7 @@ func (b *bootstrapWorker) addEvent( key model.TopicPartitionKey, row *model.RowChangedEvent, ) error { - table, ok := b.activeTables.Load(row.Table.TableID) + table, ok := b.activeTables.Load(row.PhysicalTableID) if !ok { tb := newTableStatus(key, row) b.activeTables.Store(tb.id, tb) @@ -220,7 +220,7 @@ type tableStatistic struct { func newTableStatus(key model.TopicPartitionKey, row *model.RowChangedEvent) *tableStatistic { res := &tableStatistic{ - id: row.Table.TableID, + id: row.PhysicalTableID, topic: key.Topic, } res.totalPartition.Store(key.TotalPartition) diff --git a/pkg/sink/codec/bootstraper_test.go b/pkg/sink/codec/bootstraper_test.go index f4185c8a4f2..5a1b2eb8a6c 100644 --- a/pkg/sink/codec/bootstraper_test.go +++ b/pkg/sink/codec/bootstraper_test.go @@ -29,26 +29,26 @@ func getMockTableStatus(tableName string, tableID int64, totalPartition int32, ) (model.TopicPartitionKey, *model.RowChangedEvent, *tableStatistic) { + schema := "test" tableInfo := &model.TableInfo{ + TableName: model.TableName{ + Schema: schema, + Table: tableName, + TableID: tableID, + }, TableInfo: &timodel.TableInfo{ ID: tableID, UpdateTS: 1, }, } - schema := "test" - table := &model.TableName{ - Schema: schema, - Table: tableName, - TableID: tableID, - } key := model.TopicPartitionKey{ Topic: fmt.Sprintf("%s.%s", schema, tableName), Partition: 1, TotalPartition: totalPartition, } row := &model.RowChangedEvent{ - TableInfo: tableInfo, - Table: table, + PhysicalTableID: tableID, + TableInfo: tableInfo, } tb := newTableStatus(key, row) return key, row, tb diff --git a/pkg/sink/codec/builder/codec_test.go b/pkg/sink/codec/builder/codec_test.go index 785aaeab413..3db9333e7e8 100644 --- a/pkg/sink/codec/builder/codec_test.go +++ b/pkg/sink/codec/builder/codec_test.go @@ -104,8 +104,8 @@ func TestJsonVsCraftVsPB(t *testing.T) { func codecEncodeKeyPB(event *model.RowChangedEvent) []byte { key := &benchmark.Key{ Ts: event.CommitTs, - Schema: event.Table.Schema, - Table: event.Table.Table, + Schema: event.TableInfo.GetSchemaName(), + Table: event.TableInfo.GetTableName(), RowId: event.RowID, Partition: 0, } @@ -169,8 +169,8 @@ func codecEncodeKeysPB2(events []*model.RowChangedEvent) []byte { for _, event := range events { converted.Ts = append(converted.Ts, event.CommitTs) - converted.Schema = append(converted.Schema, event.Table.Schema) - converted.Table = append(converted.Table, event.Table.Table) + converted.Schema = append(converted.Schema, event.TableInfo.GetSchemaName()) + converted.Table = append(converted.Table, event.TableInfo.GetTableName()) converted.RowId = append(converted.RowId, event.RowID) converted.Partition = append(converted.Partition, 0) } @@ -368,13 +368,15 @@ func benchmarkProtobuf1Decoding() []*model.RowChangedEvent { ev.PreColumns = codecDecodeRowChangedPB1(value.OldValue) ev.Columns = codecDecodeRowChangedPB1(value.NewValue) ev.CommitTs = key.Ts - ev.Table = &model.TableName{ - Schema: key.Schema, - Table: key.Table, + ev.TableInfo = &model.TableInfo{ + TableName: model.TableName{ + Schema: key.Schema, + Table: key.Table, + }, } if key.Partition >= 0 { - ev.Table.TableID = key.Partition - ev.Table.IsPartition = true + ev.PhysicalTableID = key.Partition + ev.TableInfo.TableName.IsPartition = true } result = append(result, ev) } @@ -425,13 +427,15 @@ func benchmarkProtobuf2Decoding() []*model.RowChangedEvent { ev.Columns = codecDecodeRowChangedPB2(values.NewValue[i]) } ev.CommitTs = ts - ev.Table = &model.TableName{ - Schema: keys.Schema[i], - Table: keys.Table[i], + ev.TableInfo = &model.TableInfo{ + TableName: model.TableName{ + Schema: keys.Schema[i], + Table: keys.Table[i], + }, } if keys.Partition[i] >= 0 { - ev.Table.TableID = keys.Partition[i] - ev.Table.IsPartition = true + ev.PhysicalTableID = keys.Partition[i] + ev.TableInfo.TableName.IsPartition = true } result = append(result, ev) } diff --git a/pkg/sink/codec/canal/canal_encoder_test.go b/pkg/sink/codec/canal/canal_encoder_test.go index e25a1517601..052b05571dd 100644 --- a/pkg/sink/codec/canal/canal_encoder_test.go +++ b/pkg/sink/codec/canal/canal_encoder_test.go @@ -31,7 +31,9 @@ var ( rowCases = [][]*model.RowChangedEvent{ {{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "t"}, + }, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, @@ -41,7 +43,9 @@ var ( { { CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "t"}, + }, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, @@ -50,8 +54,10 @@ var ( }, { CommitTs: 2, - Table: &model.TableName{Schema: "test", Table: "t"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "t"}, + }, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, }, } @@ -161,7 +167,6 @@ func TestCanalAppendRowChangedEventWithCallback(t *testing.T) { _, _, colInfo := tableInfo.GetRowColInfos() row := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, diff --git a/pkg/sink/codec/canal/canal_entry.go b/pkg/sink/codec/canal/canal_entry.go index 9d8632100b9..b5ae71396f5 100644 --- a/pkg/sink/codec/canal/canal_entry.go +++ b/pkg/sink/codec/canal/canal_entry.go @@ -190,7 +190,7 @@ func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent, onlyHandleKey // fromRowEvent builds canal entry from cdc RowChangedEvent func (b *canalEntryBuilder) fromRowEvent(e *model.RowChangedEvent, onlyHandleKeyColumns bool) (*canal.Entry, error) { eventType := convertRowEventType(e) - header := b.buildHeader(e.CommitTs, e.Table.Schema, e.Table.Table, eventType, 1) + header := b.buildHeader(e.CommitTs, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName(), eventType, 1) isDdl := isCanalDDL(eventType) // false rowData, err := b.buildRowData(e, onlyHandleKeyColumns) if err != nil { diff --git a/pkg/sink/codec/canal/canal_entry_test.go b/pkg/sink/codec/canal/canal_entry_test.go index ec0b7bf6146..93a747a3869 100644 --- a/pkg/sink/codec/canal/canal_entry_test.go +++ b/pkg/sink/codec/canal/canal_entry_test.go @@ -45,11 +45,7 @@ func TestInsert(t *testing.T) { _, _, colInfos := tableInfo.GetRowColInfos() event := &model.RowChangedEvent{ - CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "cdc", - Table: "person", - }, + CommitTs: 417318403368288260, TableInfo: tableInfo, Columns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, @@ -69,8 +65,8 @@ func TestInsert(t *testing.T) { header := entry.GetHeader() require.Equal(t, int64(1591943372224), header.GetExecuteTime()) require.Equal(t, canal.Type_MYSQL, header.GetSourceType()) - require.Equal(t, event.Table.Schema, header.GetSchemaName()) - require.Equal(t, event.Table.Table, header.GetTableName()) + require.Equal(t, event.TableInfo.GetSchemaName(), header.GetSchemaName()) + require.Equal(t, event.TableInfo.GetTableName(), header.GetTableName()) require.Equal(t, canal.EventType_INSERT, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) @@ -133,11 +129,7 @@ func TestUpdate(t *testing.T) { _, _, colInfos := tableInfo.GetRowColInfos() event := &model.RowChangedEvent{ - CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "test", - Table: "t", - }, + CommitTs: 417318403368288260, TableInfo: tableInfo, Columns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, @@ -158,8 +150,8 @@ func TestUpdate(t *testing.T) { header := entry.GetHeader() require.Equal(t, int64(1591943372224), header.GetExecuteTime()) require.Equal(t, canal.Type_MYSQL, header.GetSourceType()) - require.Equal(t, event.Table.Schema, header.GetSchemaName()) - require.Equal(t, event.Table.Table, header.GetTableName()) + require.Equal(t, event.TableInfo.GetSchemaName(), header.GetSchemaName()) + require.Equal(t, event.TableInfo.GetTableName(), header.GetTableName()) require.Equal(t, canal.EventType_UPDATE, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) @@ -222,11 +214,7 @@ func TestDelete(t *testing.T) { _, _, colInfos := tableInfo.GetRowColInfos() event := &model.RowChangedEvent{ - CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "test", - Table: "t", - }, + CommitTs: 417318403368288260, TableInfo: tableInfo, PreColumns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, @@ -239,8 +227,8 @@ func TestDelete(t *testing.T) { require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() - require.Equal(t, event.Table.Schema, header.GetSchemaName()) - require.Equal(t, event.Table.Table, header.GetTableName()) + require.Equal(t, event.TableInfo.GetSchemaName(), header.GetSchemaName()) + require.Equal(t, event.TableInfo.GetTableName(), header.GetTableName()) require.Equal(t, canal.EventType_DELETE, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) diff --git a/pkg/sink/codec/canal/canal_json_message.go b/pkg/sink/codec/canal/canal_json_message.go index 93d8ec34a82..7ac28d1f015 100644 --- a/pkg/sink/codec/canal/canal_json_message.go +++ b/pkg/sink/codec/canal/canal_json_message.go @@ -157,9 +157,11 @@ func (c *canalJSONMessageWithTiDBExtension) getCommitTs() uint64 { func canalJSONMessage2RowChange(msg canalJSONMessageInterface) (*model.RowChangedEvent, error) { result := new(model.RowChangedEvent) result.CommitTs = msg.getCommitTs() - result.Table = &model.TableName{ - Schema: *msg.getSchema(), - Table: *msg.getTable(), + result.TableInfo = &model.TableInfo{ + TableName: model.TableName{ + Schema: *msg.getSchema(), + Table: *msg.getTable(), + }, } mysqlType := msg.getMySQLType() diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder.go b/pkg/sink/codec/canal/canal_json_row_event_encoder.go index bbe4c8a1b90..25c462505c3 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder.go @@ -103,12 +103,12 @@ func newJSONMessageForDML( { const prefix string = ",\"database\":" out.RawString(prefix) - out.String(e.Table.Schema) + out.String(e.TableInfo.GetSchemaName()) } { const prefix string = ",\"table\":" out.RawString(prefix) - out.String(e.Table.Table) + out.String(e.TableInfo.GetTableName()) } { const prefix string = ",\"pkNames\":" @@ -399,8 +399,8 @@ func (c *JSONRowEventEncoder) AppendRowChangedEvent( Key: nil, Value: value, Ts: e.CommitTs, - Schema: &e.Table.Schema, - Table: &e.Table.Table, + Schema: e.TableInfo.GetSchemaNamePtr(), + Table: e.TableInfo.GetTableNamePtr(), Type: model.MessageTypeRow, Protocol: config.ProtocolCanalJSON, Callback: callback, @@ -414,7 +414,7 @@ func (c *JSONRowEventEncoder) AppendRowChangedEvent( log.Error("Single message is too large for canal-json", zap.Int("maxMessageBytes", c.config.MaxMessageBytes), zap.Int("length", originLength), - zap.Any("table", e.Table)) + zap.Any("table", e.TableInfo.TableName)) return cerror.ErrMessageTooLarge.GenWithStackByArgs() } @@ -437,14 +437,14 @@ func (c *JSONRowEventEncoder) AppendRowChangedEvent( zap.Int("maxMessageBytes", c.config.MaxMessageBytes), zap.Int("originLength", originLength), zap.Int("length", length), - zap.Any("table", e.Table)) + zap.Any("table", e.TableInfo.TableName)) return cerror.ErrMessageTooLarge.GenWithStackByArgs() } log.Warn("Single message is too large for canal-json, only encode handle-key columns", zap.Int("maxMessageBytes", c.config.MaxMessageBytes), zap.Int("originLength", originLength), zap.Int("length", length), - zap.Any("table", e.Table)) + zap.Any("table", e.TableInfo.TableName)) } if c.config.LargeMessageHandle.EnableClaimCheck() { @@ -489,7 +489,7 @@ func (c *JSONRowEventEncoder) newClaimCheckLocationMessage( log.Warn("Single message is too large for canal-json, when create the claim check location message", zap.Int("maxMessageBytes", c.config.MaxMessageBytes), zap.Int("length", length), - zap.Any("table", event.Table)) + zap.Any("table", event.TableInfo.TableName)) return nil, cerror.ErrMessageTooLarge.GenWithStackByArgs(length) } return result, nil diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go index d243501e6f2..66dfd6e9285 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go @@ -76,8 +76,8 @@ func TestDMLE2E(t *testing.T) { if enableTiDBExtension { require.Equal(t, insertEvent.CommitTs, decodedEvent.CommitTs) } - require.Equal(t, insertEvent.Table.Schema, decodedEvent.Table.Schema) - require.Equal(t, insertEvent.Table.Table, decodedEvent.Table.Table) + require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedEvent.TableInfo.GetSchemaName()) + require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedEvent.TableInfo.GetTableName()) decodedColumns := make(map[string]*model.Column, len(decodedEvent.Columns)) for _, column := range decodedEvent.Columns { @@ -157,8 +157,8 @@ func TestCanalJSONCompressionE2E(t *testing.T) { 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) + require.Equal(t, decodedEvent.TableInfo.GetSchemaName(), insertEvent.TableInfo.GetSchemaName()) + require.Equal(t, decodedEvent.TableInfo.GetTableName(), insertEvent.TableInfo.GetTableName()) // encode DDL event helper := entry.NewSchemaTestHelper(t) @@ -239,8 +239,8 @@ func TestCanalJSONClaimCheckE2E(t *testing.T) { require.NoError(t, err) require.Equal(t, insertEvent.CommitTs, decodedLargeEvent.CommitTs) - require.Equal(t, insertEvent.Table.Schema, decodedLargeEvent.Table.Schema) - require.Equal(t, insertEvent.Table.Table, decodedLargeEvent.Table.Table) + require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedLargeEvent.TableInfo.GetSchemaName()) + require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedLargeEvent.TableInfo.GetTableName()) require.Nil(t, nil, decodedLargeEvent.PreColumns) decodedColumns := make(map[string]*model.Column, len(decodedLargeEvent.Columns)) @@ -649,8 +649,8 @@ func TestCanalJSONContentCompatibleE2E(t *testing.T) { 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) + require.Equal(t, decodedEvent.TableInfo.GetSchemaName(), insertEvent.TableInfo.GetSchemaName()) + require.Equal(t, decodedEvent.TableInfo.GetTableName(), insertEvent.TableInfo.GetTableName()) obtainedColumns := make(map[string]*model.Column, len(decodedEvent.Columns)) for _, column := range decodedEvent.Columns { @@ -704,8 +704,8 @@ func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { if encodeEnable && decodeEnable { require.Equal(t, insertEvent.CommitTs, decodedEvent.CommitTs) } - require.Equal(t, insertEvent.Table.Schema, decodedEvent.Table.Schema) - require.Equal(t, insertEvent.Table.Table, decodedEvent.Table.Table) + require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedEvent.TableInfo.GetSchemaName()) + require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedEvent.TableInfo.GetTableName()) decodedColumns := make(map[string]*model.Column, len(decodedEvent.Columns)) for _, column := range decodedEvent.Columns { diff --git a/pkg/sink/codec/canal/canal_json_txn_event_encoder.go b/pkg/sink/codec/canal/canal_json_txn_event_encoder.go index b9cc734e33c..bda2df2a88b 100644 --- a/pkg/sink/codec/canal/canal_json_txn_event_encoder.go +++ b/pkg/sink/codec/canal/canal_json_txn_event_encoder.go @@ -60,7 +60,7 @@ func (j *JSONTxnEventEncoder) AppendTxnEvent( log.Warn("Single message is too large for canal-json", zap.Int("maxMessageBytes", j.config.MaxMessageBytes), zap.Int("length", length), - zap.Any("table", row.Table)) + zap.Any("table", row.TableInfo.TableName)) return cerror.ErrMessageTooLarge.GenWithStackByArgs() } j.valueBuf.Write(value) @@ -69,8 +69,8 @@ func (j *JSONTxnEventEncoder) AppendTxnEvent( } j.callback = callback j.txnCommitTs = txn.CommitTs - j.txnSchema = &txn.Table.Schema - j.txnTable = &txn.Table.Table + j.txnSchema = txn.TableInfo.GetSchemaNamePtr() + j.txnTable = txn.TableInfo.GetTableNamePtr() return nil } diff --git a/pkg/sink/codec/canal/canal_json_txn_event_encoder_test.go b/pkg/sink/codec/canal/canal_json_txn_event_encoder_test.go index 24e11efd336..d1eb0c2fe65 100644 --- a/pkg/sink/codec/canal/canal_json_txn_event_encoder_test.go +++ b/pkg/sink/codec/canal/canal_json_txn_event_encoder_test.go @@ -46,11 +46,10 @@ func TestCanalJSONTxnEventEncoderMaxMessageBytes(t *testing.T) { // the size of `testEvent` after being encoded by canal-json is 200 testEvent := &model.SingleTableTxn{ - Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Rows: []*model.RowChangedEvent{ { CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{ Name: "col1", @@ -93,11 +92,10 @@ func TestCanalJSONAppendTxnEventEncoderWithCallback(t *testing.T) { count := 0 txn := &model.SingleTableTxn{ - Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Rows: []*model.RowChangedEvent{ { CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{ Name: "a", @@ -108,7 +106,6 @@ func TestCanalJSONAppendTxnEventEncoderWithCallback(t *testing.T) { }, { CommitTs: 2, - Table: &model.TableName{Schema: "test", Table: "t"}, TableInfo: tableInfo, Columns: []*model.Column{{ Name: "a", diff --git a/pkg/sink/codec/common/message_test.go b/pkg/sink/codec/common/message_test.go index ce91cd60e1c..1e2646886a8 100644 --- a/pkg/sink/codec/common/message_test.go +++ b/pkg/sink/codec/common/message_test.go @@ -27,9 +27,11 @@ import ( func TestCreate(t *testing.T) { t.Parallel() rowEvent := &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, }, PreColumns: []*model.Column{ { @@ -46,14 +48,20 @@ func TestCreate(t *testing.T) { CommitTs: 5678, } - msg := NewMsg(config.ProtocolOpen, []byte("key1"), []byte("value1"), rowEvent.CommitTs, model.MessageTypeRow, &rowEvent.Table.Schema, &rowEvent.Table.Table) + msg := NewMsg(config.ProtocolOpen, + []byte("key1"), + []byte("value1"), + rowEvent.CommitTs, + model.MessageTypeRow, + &rowEvent.TableInfo.TableName.Schema, + &rowEvent.TableInfo.TableName.Table) require.Equal(t, []byte("key1"), msg.Key) require.Equal(t, []byte("value1"), msg.Value) require.Equal(t, rowEvent.CommitTs, msg.Ts) require.Equal(t, model.MessageTypeRow, msg.Type) - require.Equal(t, rowEvent.Table.Schema, *msg.Schema) - require.Equal(t, rowEvent.Table.Table, *msg.Table) + require.Equal(t, rowEvent.TableInfo.GetSchemaName(), *msg.Schema) + require.Equal(t, rowEvent.TableInfo.GetTableName(), *msg.Table) require.Equal(t, config.ProtocolOpen, msg.Protocol) ft := types.NewFieldType(0) diff --git a/pkg/sink/codec/craft/craft_decoder.go b/pkg/sink/codec/craft/craft_decoder.go index 18957b3be87..734a9b98bb0 100644 --- a/pkg/sink/codec/craft/craft_decoder.go +++ b/pkg/sink/codec/craft/craft_decoder.go @@ -76,14 +76,17 @@ func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { } } ev.CommitTs = b.headers.GetTs(b.index) - ev.Table = &model.TableName{ - Schema: b.headers.GetSchema(b.index), - Table: b.headers.GetTable(b.index), + ev.TableInfo = &model.TableInfo{ + TableName: model.TableName{ + Schema: b.headers.GetSchema(b.index), + Table: b.headers.GetTable(b.index), + IsPartition: false, + }, } partition := b.headers.GetPartition(b.index) if partition >= 0 { - ev.Table.TableID = partition - ev.Table.IsPartition = true + ev.PhysicalTableID = partition + ev.TableInfo.TableName.IsPartition = true } b.index++ return ev, nil diff --git a/pkg/sink/codec/craft/craft_encoder_test.go b/pkg/sink/codec/craft/craft_encoder_test.go index 4fd39960554..491b0da67ed 100644 --- a/pkg/sink/codec/craft/craft_encoder_test.go +++ b/pkg/sink/codec/craft/craft_encoder_test.go @@ -33,7 +33,9 @@ func TestCraftMaxMessageBytes(t *testing.T) { testEvent := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "b"}, + }, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, @@ -60,7 +62,9 @@ func TestCraftMaxBatchSize(t *testing.T) { testEvent := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "b"}, + }, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, @@ -220,7 +224,9 @@ func TestCraftAppendRowChangedEventWithCallback(t *testing.T) { row := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "b"}, + }, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, diff --git a/pkg/sink/codec/craft/model.go b/pkg/sink/codec/craft/model.go index 71af0e88f14..3aa1945d547 100644 --- a/pkg/sink/codec/craft/model.go +++ b/pkg/sink/codec/craft/model.go @@ -460,16 +460,16 @@ func (b *RowChangedEventBuffer) Encode() []byte { // AppendRowChangedEvent append a new event to buffer func (b *RowChangedEventBuffer) AppendRowChangedEvent(ev *model.RowChangedEvent, onlyHandleKeyColumns bool) (rows, size int) { var partition int64 = -1 - if ev.Table.IsPartition { - partition = ev.Table.TableID + if ev.TableInfo.IsPartitionTable() { + partition = ev.PhysicalTableID } var schema, table *string - if len(ev.Table.Schema) > 0 { - schema = &ev.Table.Schema + if len(ev.TableInfo.GetSchemaName()) > 0 { + schema = ev.TableInfo.GetSchemaNamePtr() } - if len(ev.Table.Table) > 0 { - table = &ev.Table.Table + if len(ev.TableInfo.GetTableName()) > 0 { + table = ev.TableInfo.GetTableNamePtr() } b.estimatedSize += b.headers.appendHeader( diff --git a/pkg/sink/codec/csv/csv_encoder_test.go b/pkg/sink/codec/csv/csv_encoder_test.go index 783dbfa43a4..1ff4fdf1927 100644 --- a/pkg/sink/codec/csv/csv_encoder_test.go +++ b/pkg/sink/codec/csv/csv_encoder_test.go @@ -25,13 +25,16 @@ import ( ) func TestCSVBatchCodec(t *testing.T) { + tableInfo := &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "table1"}, + } testCases := []*model.SingleTableTxn{ { - Table: &model.TableName{Schema: "test", Table: "table1"}, + TableInfo: tableInfo, Rows: []*model.RowChangedEvent{ { - CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, + CommitTs: 1, + TableInfo: tableInfo, Columns: []*model.Column{{ Name: "tiny", Value: int64(1), Type: mysql.TypeTiny, @@ -44,8 +47,8 @@ func TestCSVBatchCodec(t *testing.T) { }}, }, { - CommitTs: 2, - Table: &model.TableName{Schema: "test", Table: "table1"}, + CommitTs: 2, + TableInfo: tableInfo, Columns: []*model.Column{{ Name: "tiny", Value: int64(2), Type: mysql.TypeTiny, @@ -60,8 +63,8 @@ func TestCSVBatchCodec(t *testing.T) { }, }, { - Table: &model.TableName{Schema: "test", Table: "table1"}, - Rows: nil, + TableInfo: tableInfo, + Rows: nil, }, } @@ -98,8 +101,10 @@ func TestCSVAppendRowChangedEventWithCallback(t *testing.T) { count := 0 row := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, - Columns: []*model.Column{{Name: "tiny", Value: int64(1), Type: mysql.TypeTiny}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "table1"}, + }, + Columns: []*model.Column{{Name: "tiny", Value: int64(1), Type: mysql.TypeTiny}}, ColInfos: []rowcodec.ColInfo{{ ID: 1, IsPKHandle: false, @@ -109,8 +114,8 @@ func TestCSVAppendRowChangedEventWithCallback(t *testing.T) { } txn := &model.SingleTableTxn{ - Table: row.Table, - Rows: []*model.RowChangedEvent{row}, + TableInfo: row.TableInfo, + Rows: []*model.RowChangedEvent{row}, } callback := func() { count += 1 diff --git a/pkg/sink/codec/csv/csv_message.go b/pkg/sink/codec/csv/csv_message.go index dedf6c70820..9fa17ceecfe 100644 --- a/pkg/sink/codec/csv/csv_message.go +++ b/pkg/sink/codec/csv/csv_message.go @@ -355,8 +355,8 @@ func rowChangedEvent2CSVMsg(csvConfig *common.Config, e *model.RowChangedEvent) csvMsg := &csvMessage{ config: csvConfig, - tableName: e.Table.Table, - schemaName: e.Table.Schema, + tableName: e.TableInfo.GetTableName(), + schemaName: e.TableInfo.GetSchemaName(), commitTs: e.CommitTs, newRecord: true, } @@ -407,9 +407,11 @@ func csvMsg2RowChangedEvent(csvConfig *common.Config, csvMsg *csvMessage, ticols e := new(model.RowChangedEvent) e.CommitTs = csvMsg.commitTs - e.Table = &model.TableName{ - Schema: csvMsg.schemaName, - Table: csvMsg.tableName, + e.TableInfo = &model.TableInfo{ + TableName: model.TableName{ + Schema: csvMsg.schemaName, + Table: csvMsg.tableName, + }, } if csvMsg.opType == operationDelete { e.PreColumns, err = csvColumns2RowChangeColumns(csvConfig, csvMsg.columns, ticols) diff --git a/pkg/sink/codec/csv/csv_message_test.go b/pkg/sink/codec/csv/csv_message_test.go index 5c0baf2a116..faee6561a0c 100644 --- a/pkg/sink/codec/csv/csv_message_test.go +++ b/pkg/sink/codec/csv/csv_message_test.go @@ -955,9 +955,11 @@ func TestRowChangeEventConversion(t *testing.T) { colInfos = append(colInfos, c.colInfo) } row.ColInfos = colInfos - row.Table = &model.TableName{ - Table: fmt.Sprintf("table%d", idx), - Schema: "test", + row.TableInfo = &model.TableInfo{ + TableName: model.TableName{ + Table: fmt.Sprintf("table%d", idx), + Schema: "test", + }, } if idx%3 == 0 { // delete operation diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index f28c5971e30..a1e0c1a8173 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -538,8 +538,8 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteInt64Field("ts_ms", commitTime.UnixMilli()) // snapshot field is a string of true,last,false,incremental jWriter.WriteStringField("snapshot", "false") - jWriter.WriteStringField("db", e.Table.Schema) - jWriter.WriteStringField("table", e.Table.Table) + jWriter.WriteStringField("db", e.TableInfo.GetSchemaName()) + jWriter.WriteStringField("table", e.TableInfo.GetTableName()) jWriter.WriteInt64Field("server_id", 0) jWriter.WriteNullField("gtid") jWriter.WriteStringField("file", "") @@ -595,7 +595,10 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteObjectField("schema", func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Envelope", c.clusterID, e.Table.Schema, e.Table.Table)) + jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Envelope", + c.clusterID, + e.TableInfo.GetSchemaName(), + e.TableInfo.GetTableName())) jWriter.WriteIntField("version", 1) jWriter.WriteArrayField("fields", func() { // schema is the same for `before` and `after`. So we build a new buffer to @@ -621,7 +624,10 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Value", c.clusterID, e.Table.Schema, e.Table.Table)) + jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Value", + c.clusterID, + e.TableInfo.GetSchemaName(), + e.TableInfo.GetTableName())) jWriter.WriteStringField("field", "before") jWriter.WriteArrayField("fields", func() { jWriter.WriteRaw(fieldsJSON) @@ -630,7 +636,10 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Value", c.clusterID, e.Table.Schema, e.Table.Table)) + jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Value", + c.clusterID, + e.TableInfo.GetSchemaName(), + e.TableInfo.GetTableName())) jWriter.WriteStringField("field", "after") jWriter.WriteArrayField("fields", func() { jWriter.WriteRaw(fieldsJSON) diff --git a/pkg/sink/codec/debezium/codec_test.go b/pkg/sink/codec/debezium/codec_test.go index 87e9d869bc8..4371d3f89a4 100644 --- a/pkg/sink/codec/debezium/codec_test.go +++ b/pkg/sink/codec/debezium/codec_test.go @@ -38,7 +38,9 @@ func TestEncodeInsert(t *testing.T) { e := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "table1"}, + }, Columns: []*model.Column{{ Name: "tiny", Value: int64(1), Type: mysql.TypeTiny, @@ -202,7 +204,9 @@ func TestEncodeUpdate(t *testing.T) { e := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "table1"}, + }, Columns: []*model.Column{{ Name: "tiny", Value: int64(1), Type: mysql.TypeTiny, @@ -372,7 +376,9 @@ func TestEncodeDelete(t *testing.T) { e := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "table1"}, + }, PreColumns: []*model.Column{{ Name: "tiny", Value: int64(2), Type: mysql.TypeTiny, @@ -536,7 +542,9 @@ func BenchmarkEncodeOneTinyColumn(b *testing.B) { e := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "table1"}, + }, Columns: []*model.Column{{ Name: "tiny", Value: int64(10), Type: mysql.TypeTiny, @@ -568,7 +576,9 @@ func BenchmarkEncodeLargeText(b *testing.B) { e := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "table1"}, + }, Columns: []*model.Column{{ Name: "str", Value: []byte(randstr.String(1024)), Type: mysql.TypeVarchar, @@ -600,7 +610,9 @@ func BenchmarkEncodeLargeBinary(b *testing.B) { e := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "test", Table: "table1"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "test", Table: "table1"}, + }, Columns: []*model.Column{{ Name: "bin", Value: []byte(randstr.String(1024)), Type: mysql.TypeVarchar, Flag: model.BinaryFlag, diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go index a7ef32a0d3e..4dcb1960e8f 100644 --- a/pkg/sink/codec/debezium/encoder.go +++ b/pkg/sink/codec/debezium/encoder.go @@ -64,8 +64,8 @@ func (d *BatchEncoder) AppendRowChangedEvent( Key: nil, Value: value, Ts: e.CommitTs, - Schema: &e.Table.Schema, - Table: &e.Table.Table, + Schema: e.TableInfo.GetSchemaNamePtr(), + Table: e.TableInfo.GetTableNamePtr(), Type: model.MessageTypeRow, Protocol: config.ProtocolDebezium, Callback: callback, diff --git a/pkg/sink/codec/internal/batch_tester.go b/pkg/sink/codec/internal/batch_tester.go index 8006bc21a00..acb5c342e25 100644 --- a/pkg/sink/codec/internal/batch_tester.go +++ b/pkg/sink/codec/internal/batch_tester.go @@ -25,10 +25,15 @@ import ( ) var ( + tableInfo = &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", Table: "b", + }, + } // CodecRowCases defines test cases for RowChangedEvent. CodecRowCases = [][]*model.RowChangedEvent{{{ - CommitTs: 424316552636792833, - Table: &model.TableName{Schema: "a", Table: "b"}, + CommitTs: 424316552636792833, + TableInfo: tableInfo, PreColumns: []*model.Column{ {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, @@ -50,8 +55,8 @@ var ( {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }}, {{ - CommitTs: 424316553934667777, - Table: &model.TableName{Schema: "a", Table: "c"}, + CommitTs: 424316553934667777, + TableInfo: tableInfo, PreColumns: []*model.Column{ {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, @@ -73,8 +78,8 @@ var ( {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }, { - CommitTs: 424316554327097345, - Table: &model.TableName{Schema: "a", Table: "d"}, + CommitTs: 424316554327097345, + TableInfo: tableInfo, PreColumns: []*model.Column{ {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, @@ -96,8 +101,8 @@ var ( {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }, { - CommitTs: 424316554746789889, - Table: &model.TableName{Schema: "a", Table: "e"}, + CommitTs: 424316554746789889, + TableInfo: tableInfo, PreColumns: []*model.Column{ {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, @@ -119,8 +124,8 @@ var ( {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }, { - CommitTs: 424316555073945601, - Table: &model.TableName{Schema: "a", Table: "f", TableID: 6, IsPartition: true}, + CommitTs: 424316555073945601, + TableInfo: tableInfo, PreColumns: []*model.Column{ {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, @@ -145,41 +150,25 @@ var ( // CodecDDLCases defines test cases for DDLEvent. CodecDDLCases = [][]*model.DDLEvent{{{ - CommitTs: 424316555979653121, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table a", - Type: 1, + CommitTs: 424316555979653121, + TableInfo: tableInfo, + Query: "create table a", + Type: 1, }}, {{ - CommitTs: 424316583965360129, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table a", - Type: 1, + CommitTs: 424316583965360129, + TableInfo: tableInfo, + Query: "create table a", + Type: 1, }, { - CommitTs: 424316586087940097, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table b", - Type: 2, + CommitTs: 424316586087940097, + TableInfo: tableInfo, + Query: "create table b", + Type: 2, }, { - CommitTs: 424316588736118785, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table c", - Type: 3, + CommitTs: 424316588736118785, + TableInfo: tableInfo, + Query: "create table c", + Type: 3, }}, {}} // CodecResolvedTSCases defines test cases for resolved ts events. diff --git a/pkg/sink/codec/maxwell/maxwell_encoder_test.go b/pkg/sink/codec/maxwell/maxwell_encoder_test.go index d8d24deaad5..d83481b6fb4 100644 --- a/pkg/sink/codec/maxwell/maxwell_encoder_test.go +++ b/pkg/sink/codec/maxwell/maxwell_encoder_test.go @@ -30,8 +30,10 @@ func TestMaxwellBatchCodec(t *testing.T) { rowCases := [][]*model.RowChangedEvent{{{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 3, Value: 10}}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "b"}, + }, + Columns: []*model.Column{{Name: "col1", Type: 3, Value: 10}}, }}, {}} for _, cs := range rowCases { encoder := newEncoder(&common.Config{}) @@ -77,7 +79,9 @@ func TestMaxwellAppendRowChangedEventWithCallback(t *testing.T) { row := &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "b"}, + }, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, diff --git a/pkg/sink/codec/maxwell/maxwell_message.go b/pkg/sink/codec/maxwell/maxwell_message.go index 84593bf90c4..9033a2be469 100644 --- a/pkg/sink/codec/maxwell/maxwell_message.go +++ b/pkg/sink/codec/maxwell/maxwell_message.go @@ -45,20 +45,20 @@ func (m *maxwellMessage) encode() ([]byte, error) { func rowChangeToMaxwellMsg(e *model.RowChangedEvent, onlyHandleKeyColumns bool) (*internal.MessageKey, *maxwellMessage) { var partition *int64 - if e.Table.IsPartition { - partition = &e.Table.TableID + if e.TableInfo.IsPartitionTable() { + partition = &e.PhysicalTableID } key := &internal.MessageKey{ Ts: e.CommitTs, - Schema: e.Table.Schema, - Table: e.Table.Table, + Schema: e.TableInfo.GetSchemaName(), + Table: e.TableInfo.GetTableName(), Partition: partition, Type: model.MessageTypeRow, } value := &maxwellMessage{ Ts: 0, - Database: e.Table.Schema, - Table: e.Table.Table, + Database: e.TableInfo.GetSchemaName(), + Table: e.TableInfo.GetTableName(), Data: make(map[string]interface{}), Old: make(map[string]interface{}), } diff --git a/pkg/sink/codec/maxwell/maxwell_message_test.go b/pkg/sink/codec/maxwell/maxwell_message_test.go index 585c332b650..d932247442f 100644 --- a/pkg/sink/codec/maxwell/maxwell_message_test.go +++ b/pkg/sink/codec/maxwell/maxwell_message_test.go @@ -50,7 +50,9 @@ func TestEncodeBinaryToMaxwell(t *testing.T) { } e := &model.RowChangedEvent{ - Table: &model.TableName{Schema: "a", Table: "b"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "b"}, + }, Columns: []*model.Column{column}, } diff --git a/pkg/sink/codec/open/open_protocol_decoder.go b/pkg/sink/codec/open/open_protocol_decoder.go index f56a978a351..d790208f6a8 100644 --- a/pkg/sink/codec/open/open_protocol_decoder.go +++ b/pkg/sink/codec/open/open_protocol_decoder.go @@ -254,8 +254,8 @@ func (b *BatchDecoder) assembleHandleKeyOnlyEvent( ctx context.Context, handleKeyOnlyEvent *model.RowChangedEvent, ) (*model.RowChangedEvent, error) { var ( - schema = handleKeyOnlyEvent.Table.Schema - table = handleKeyOnlyEvent.Table.Table + schema = handleKeyOnlyEvent.TableInfo.GetSchemaName() + table = handleKeyOnlyEvent.TableInfo.GetTableName() commitTs = handleKeyOnlyEvent.CommitTs ) diff --git a/pkg/sink/codec/open/open_protocol_encoder.go b/pkg/sink/codec/open/open_protocol_encoder.go index 03eb9e981d2..fe8fb752a3f 100644 --- a/pkg/sink/codec/open/open_protocol_encoder.go +++ b/pkg/sink/codec/open/open_protocol_encoder.go @@ -69,7 +69,7 @@ func (d *BatchEncoder) buildMessageOnlyHandleKeyColumns(e *model.RowChangedEvent log.Warn("Single message is too large for open-protocol, only encode handle key columns", zap.Int("maxMessageBytes", d.config.MaxMessageBytes), zap.Int("length", length), - zap.Any("table", e.Table), + zap.Any("table", e.TableInfo.TableName), zap.Any("key", key)) return nil, nil, cerror.ErrMessageTooLarge.GenWithStackByArgs() } @@ -112,7 +112,7 @@ func (d *BatchEncoder) AppendRowChangedEvent( log.Warn("Single message is too large for open-protocol", zap.Int("maxMessageBytes", d.config.MaxMessageBytes), zap.Int("length", length), - zap.Any("table", e.Table), + zap.Any("table", e.TableInfo.TableName), zap.Any("key", key)) return cerror.ErrMessageTooLarge.GenWithStackByArgs() } @@ -168,8 +168,8 @@ func (d *BatchEncoder) AppendRowChangedEvent( message.Value = append(message.Value, valueLenByte[:]...) message.Value = append(message.Value, value...) message.Ts = e.CommitTs - message.Schema = &e.Table.Schema - message.Table = &e.Table.Table + message.Schema = e.TableInfo.GetSchemaNamePtr() + message.Table = e.TableInfo.GetTableNamePtr() message.IncRowsCount() if callback != nil { diff --git a/pkg/sink/codec/open/open_protocol_encoder_test.go b/pkg/sink/codec/open/open_protocol_encoder_test.go index 7758112a8bc..6e845f35113 100644 --- a/pkg/sink/codec/open/open_protocol_encoder_test.go +++ b/pkg/sink/codec/open/open_protocol_encoder_test.go @@ -42,7 +42,9 @@ func TestBuildOpenProtocolBatchEncoder(t *testing.T) { var ( testEvent = &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "b"}, + }, Columns: []*model.Column{ { Name: "col1", @@ -59,7 +61,9 @@ var ( } largeTestEvent = &model.RowChangedEvent{ CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + TableInfo: &model.TableInfo{ + TableName: model.TableName{Schema: "a", Table: "b"}, + }, Columns: []*model.Column{ { Name: "col1", @@ -497,7 +501,7 @@ func TestE2EClaimCheckMessage(t *testing.T) { require.NoError(t, err) require.Equal(t, largeTestEvent.CommitTs, decodedLargeEvent.CommitTs) - require.Equal(t, largeTestEvent.Table, decodedLargeEvent.Table) + require.Equal(t, largeTestEvent.TableInfo.GetTableName(), decodedLargeEvent.TableInfo.GetTableName()) decodedColumns := make(map[string]*model.Column, len(decodedLargeEvent.Columns)) for _, column := range decodedLargeEvent.Columns { diff --git a/pkg/sink/codec/open/open_protocol_message.go b/pkg/sink/codec/open/open_protocol_message.go index 6bb710c1c1f..a2d96781089 100644 --- a/pkg/sink/codec/open/open_protocol_message.go +++ b/pkg/sink/codec/open/open_protocol_message.go @@ -101,13 +101,13 @@ func rowChangeToMsg( config *common.Config, largeMessageOnlyHandleKeyColumns bool) (*internal.MessageKey, *messageRow, error) { var partition *int64 - if e.Table.IsPartition { - partition = &e.Table.TableID + if e.TableInfo.IsPartitionTable() { + partition = &e.PhysicalTableID } key := &internal.MessageKey{ Ts: e.CommitTs, - Schema: e.Table.Schema, - Table: e.Table.Table, + Schema: e.TableInfo.GetSchemaName(), + Table: e.TableInfo.GetTableName(), RowID: e.RowID, Partition: partition, Type: model.MessageTypeRow, @@ -144,14 +144,16 @@ func msgToRowChange(key *internal.MessageKey, value *messageRow) *model.RowChang // TODO: we lost the startTs from kafka message // startTs-based txn filter is out of work e.CommitTs = key.Ts - e.Table = &model.TableName{ - Schema: key.Schema, - Table: key.Table, + e.TableInfo = &model.TableInfo{ + TableName: model.TableName{ + Schema: key.Schema, + Table: key.Table, + }, } // TODO: we lost the tableID from kafka message if key.Partition != nil { - e.Table.TableID = *key.Partition - e.Table.IsPartition = true + e.PhysicalTableID = *key.Partition + e.TableInfo.TableName.IsPartition = true } if len(value.Delete) != 0 { diff --git a/pkg/sink/codec/open/open_protocol_message_test.go b/pkg/sink/codec/open/open_protocol_message_test.go index 89dd1bf7a0b..a4e55fb9312 100644 --- a/pkg/sink/codec/open/open_protocol_message_test.go +++ b/pkg/sink/codec/open/open_protocol_message_test.go @@ -100,13 +100,15 @@ func TestOnlyOutputUpdatedColumn(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolOpen) codecConfig.OnlyOutputUpdatedColumns = true - table := &model.TableName{ - Schema: "test", - Table: "test", + tableInfo := &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "test", + }, } event := &model.RowChangedEvent{ - Table: table, + TableInfo: tableInfo, PreColumns: []*model.Column{ { Name: "test", @@ -128,7 +130,7 @@ func TestOnlyOutputUpdatedColumn(t *testing.T) { require.False(t, ok) event = &model.RowChangedEvent{ - Table: table, + TableInfo: tableInfo, PreColumns: []*model.Column{ { Name: "test", @@ -149,7 +151,7 @@ func TestOnlyOutputUpdatedColumn(t *testing.T) { // column type updated, so output it. event = &model.RowChangedEvent{ - Table: table, + TableInfo: tableInfo, PreColumns: []*model.Column{ { Name: "test", @@ -169,7 +171,7 @@ func TestOnlyOutputUpdatedColumn(t *testing.T) { require.True(t, ok) event = &model.RowChangedEvent{ - Table: table, + TableInfo: tableInfo, PreColumns: []*model.Column{ { Name: "test", @@ -194,9 +196,11 @@ func TestRowChanged2MsgOnlyHandleKeyColumns(t *testing.T) { insertEvent := &model.RowChangedEvent{ CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "schema", - Table: "table", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "schema", + Table: "table", + }, }, Columns: []*model.Column{ {Name: "id", Flag: model.HandleKeyFlag, Type: mysql.TypeLonglong, Value: 1}, @@ -221,9 +225,11 @@ func TestRowChanged2MsgOnlyHandleKeyColumns(t *testing.T) { insertEventNoHandleKey := &model.RowChangedEvent{ CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "schema", - Table: "table", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "schema", + Table: "table", + }, }, Columns: []*model.Column{ {Name: "id", Type: mysql.TypeLonglong, Value: 1}, @@ -235,9 +241,11 @@ func TestRowChanged2MsgOnlyHandleKeyColumns(t *testing.T) { updateEvent := &model.RowChangedEvent{ CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "schema", - Table: "table", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "schema", + Table: "table", + }, }, Columns: []*model.Column{ {Name: "id", Flag: model.HandleKeyFlag, Type: mysql.TypeLonglong, Value: 1}, @@ -263,9 +271,11 @@ func TestRowChanged2MsgOnlyHandleKeyColumns(t *testing.T) { updateEventNoHandleKey := &model.RowChangedEvent{ CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "schema", - Table: "table", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "schema", + Table: "table", + }, }, Columns: []*model.Column{ {Name: "id", Type: mysql.TypeLonglong, Value: 1}, @@ -281,9 +291,11 @@ func TestRowChanged2MsgOnlyHandleKeyColumns(t *testing.T) { deleteEvent := &model.RowChangedEvent{ CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "schema", - Table: "table", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "schema", + Table: "table", + }, }, PreColumns: []*model.Column{ {Name: "id", Flag: model.HandleKeyFlag, Type: mysql.TypeLonglong, Value: 1}, @@ -308,9 +320,11 @@ func TestRowChanged2MsgOnlyHandleKeyColumns(t *testing.T) { deleteEventNoHandleKey := &model.RowChangedEvent{ CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "schema", - Table: "table", + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "schema", + Table: "table", + }, }, PreColumns: []*model.Column{ {Name: "id", Type: mysql.TypeLonglong, Value: 1}, diff --git a/pkg/sink/codec/simple/avro.go b/pkg/sink/codec/simple/avro.go index 881e86469cc..0a0fddcff05 100644 --- a/pkg/sink/codec/simple/avro.go +++ b/pkg/sink/codec/simple/avro.go @@ -235,8 +235,8 @@ func newDMLMessageMap( ) (map[string]interface{}, error) { m := map[string]interface{}{ "version": defaultVersion, - "database": event.Table.Schema, - "table": event.Table.Table, + "database": event.TableInfo.GetSchemaName(), + "table": event.TableInfo.GetTableName(), "tableID": event.TableInfo.ID, "commitTs": int64(event.CommitTs), "buildTs": time.Now().UnixMilli(), diff --git a/pkg/sink/codec/simple/encoder.go b/pkg/sink/codec/simple/encoder.go index db5671600cc..7cd44c012f7 100644 --- a/pkg/sink/codec/simple/encoder.go +++ b/pkg/sink/codec/simple/encoder.go @@ -53,8 +53,8 @@ func (e *encoder) AppendRowChangedEvent( result := &common.Message{ Value: value, Ts: event.CommitTs, - Schema: &event.Table.Schema, - Table: &event.Table.Table, + Schema: event.TableInfo.GetSchemaNamePtr(), + Table: event.TableInfo.GetTableNamePtr(), Type: model.MessageTypeRow, Protocol: config.ProtocolSimple, Callback: callback, @@ -70,7 +70,7 @@ func (e *encoder) AppendRowChangedEvent( log.Error("Single message is too large for simple", zap.Int("maxMessageBytes", e.config.MaxMessageBytes), zap.Int("length", result.Length()), - zap.Any("table", event.Table)) + zap.Any("table", event.TableInfo.TableName)) return cerror.ErrMessageTooLarge.GenWithStackByArgs() } @@ -99,7 +99,7 @@ func (e *encoder) AppendRowChangedEvent( zap.Int("maxMessageBytes", e.config.MaxMessageBytes), zap.Int("originLength", result.Length()), zap.Int("length", result.Length()), - zap.Any("table", event.Table)) + zap.Any("table", event.TableInfo.TableName)) e.messages = append(e.messages, result) return nil } @@ -107,7 +107,7 @@ func (e *encoder) AppendRowChangedEvent( log.Error("Single message is still too large for simple after only encode handle key columns", zap.Int("maxMessageBytes", e.config.MaxMessageBytes), zap.Int("length", result.Length()), - zap.Any("table", event.Table)) + zap.Any("table", event.TableInfo.TableName)) return cerror.ErrMessageTooLarge.GenWithStackByArgs() } diff --git a/pkg/sink/codec/simple/encoder_test.go b/pkg/sink/codec/simple/encoder_test.go index a8349c34b6b..e8e1adde714 100644 --- a/pkg/sink/codec/simple/encoder_test.go +++ b/pkg/sink/codec/simple/encoder_test.go @@ -243,8 +243,8 @@ func TestEncodeDDLEvent(t *testing.T) { decodedRow, err := dec.NextRowChangedEvent() require.NoError(t, err) require.Equal(t, decodedRow.CommitTs, insertEvent.CommitTs) - require.Equal(t, decodedRow.Table.Schema, insertEvent.Table.Schema) - require.Equal(t, decodedRow.Table.Table, insertEvent.Table.Table) + require.Equal(t, decodedRow.TableInfo.GetSchemaName(), insertEvent.TableInfo.GetSchemaName()) + require.Equal(t, decodedRow.TableInfo.GetTableName(), insertEvent.TableInfo.GetTableName()) require.Nil(t, decodedRow.PreColumns) m, err = enc.EncodeDDLEvent(renameTableDDLEvent) @@ -293,8 +293,8 @@ func TestEncodeDDLEvent(t *testing.T) { decodedRow, err = dec.NextRowChangedEvent() require.NoError(t, err) require.Equal(t, decodedRow.CommitTs, insertEvent2.CommitTs) - require.Equal(t, decodedRow.Table.Schema, insertEvent2.Table.Schema) - require.Equal(t, decodedRow.Table.Table, insertEvent2.Table.Table) + require.Equal(t, decodedRow.TableInfo.GetSchemaName(), insertEvent2.TableInfo.GetSchemaName()) + require.Equal(t, decodedRow.TableInfo.GetTableName(), insertEvent2.TableInfo.GetTableName()) require.Nil(t, decodedRow.PreColumns) } } @@ -554,8 +554,8 @@ func TestEncodeBootstrapEvent(t *testing.T) { decodedRow, err := dec.NextRowChangedEvent() require.NoError(t, err) require.Equal(t, decodedRow.CommitTs, row.CommitTs) - require.Equal(t, decodedRow.Table.Schema, row.Table.Schema) - require.Equal(t, decodedRow.Table.Table, row.Table.Table) + require.Equal(t, decodedRow.TableInfo.GetSchemaName(), row.TableInfo.GetSchemaName()) + require.Equal(t, decodedRow.TableInfo.GetTableName(), row.TableInfo.GetTableName()) require.Nil(t, decodedRow.PreColumns) } } @@ -634,9 +634,9 @@ func TestEncodeLargeEventsNormal(t *testing.T) { require.NoError(t, err) require.Equal(t, decodedRow.CommitTs, event.CommitTs) - require.Equal(t, decodedRow.Table.Schema, event.Table.Schema) - require.Equal(t, decodedRow.Table.Table, event.Table.Table) - require.Equal(t, decodedRow.Table.TableID, event.Table.TableID) + require.Equal(t, decodedRow.TableInfo.GetSchemaName(), event.TableInfo.GetSchemaName()) + require.Equal(t, decodedRow.TableInfo.GetTableName(), event.TableInfo.GetTableName()) + require.Equal(t, decodedRow.PhysicalTableID, event.PhysicalTableID) decodedColumns := make(map[string]*model.Column, len(decodedRow.Columns)) for _, column := range decodedRow.Columns { @@ -767,8 +767,8 @@ func TestLargerMessageHandleClaimCheck(t *testing.T) { require.NoError(t, err) require.Equal(t, decodedRow.CommitTs, updateEvent.CommitTs) - require.Equal(t, decodedRow.Table.Schema, updateEvent.Table.Schema) - require.Equal(t, decodedRow.Table.Table, updateEvent.Table.Table) + require.Equal(t, decodedRow.TableInfo.GetSchemaName(), updateEvent.TableInfo.GetSchemaName()) + require.Equal(t, decodedRow.TableInfo.GetTableName(), updateEvent.TableInfo.GetTableName()) decodedColumns := make(map[string]*model.Column, len(decodedRow.Columns)) for _, column := range decodedRow.Columns { diff --git a/pkg/sink/codec/simple/message.go b/pkg/sink/codec/simple/message.go index da45db63fe9..27a6ff6dab2 100644 --- a/pkg/sink/codec/simple/message.go +++ b/pkg/sink/codec/simple/message.go @@ -384,13 +384,9 @@ func buildRowChangedEvent( msg *message, tableInfo *model.TableInfo, enableRowChecksum bool, ) (*model.RowChangedEvent, error) { result := &model.RowChangedEvent{ - CommitTs: msg.CommitTs, - Table: &model.TableName{ - Schema: msg.Schema, - Table: msg.Table, - TableID: msg.TableID, - }, - TableInfo: tableInfo, + CommitTs: msg.CommitTs, + PhysicalTableID: msg.TableID, + TableInfo: tableInfo, } columns, err := decodeColumns(msg.Data, tableInfo.Columns) @@ -578,8 +574,8 @@ func newDMLMessage( ) (*message, error) { m := &message{ Version: defaultVersion, - Schema: event.Table.Schema, - Table: event.Table.Table, + Schema: event.TableInfo.GetSchemaName(), + Table: event.TableInfo.GetTableName(), TableID: event.TableInfo.ID, CommitTs: event.CommitTs, BuildTs: time.Now().UnixMilli(),