github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/model/sink.go (about)

     1  // Copyright 2020 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package model
    15  
    16  import (
    17  	"fmt"
    18  	"sort"
    19  	"strconv"
    20  	"strings"
    21  	"sync/atomic"
    22  	"unsafe"
    23  
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/tidb/pkg/kv"
    26  	"github.com/pingcap/tidb/pkg/parser/model"
    27  	"github.com/pingcap/tidb/pkg/parser/mysql"
    28  	"github.com/pingcap/tidb/pkg/util/rowcodec"
    29  	"github.com/pingcap/tiflow/pkg/errors"
    30  	"github.com/pingcap/tiflow/pkg/integrity"
    31  	"github.com/pingcap/tiflow/pkg/quotes"
    32  	"github.com/pingcap/tiflow/pkg/sink"
    33  	"github.com/pingcap/tiflow/pkg/util"
    34  	"go.uber.org/zap"
    35  )
    36  
    37  //go:generate msgp
    38  
    39  // MessageType is the type of message, which is used by MqSink and RedoLog.
    40  type MessageType int
    41  
    42  const (
    43  	// MessageTypeUnknown is unknown type of message key
    44  	MessageTypeUnknown MessageType = iota
    45  	// MessageTypeRow is row type of message key
    46  	MessageTypeRow
    47  	// MessageTypeDDL is ddl type of message key
    48  	MessageTypeDDL
    49  	// MessageTypeResolved is resolved type of message key
    50  	MessageTypeResolved
    51  )
    52  
    53  const (
    54  	// the RowChangedEvent order in the same transaction
    55  	typeDelete = iota + 1
    56  	typeUpdate
    57  	typeInsert
    58  )
    59  
    60  // ColumnFlagType is for encapsulating the flag operations for different flags.
    61  type ColumnFlagType util.Flag
    62  
    63  const (
    64  	// BinaryFlag means the column charset is binary
    65  	BinaryFlag ColumnFlagType = 1 << ColumnFlagType(iota)
    66  	// HandleKeyFlag means the column is selected as the handle key
    67  	// The handleKey is chosen by the following rules in the order:
    68  	// 1. if the table has primary key, it's the handle key.
    69  	// 2. If the table has not null unique key, it's the handle key.
    70  	// 3. If the table has no primary key and no not null unique key, it has no handleKey.
    71  	HandleKeyFlag
    72  	// GeneratedColumnFlag means the column is a generated column
    73  	GeneratedColumnFlag
    74  	// PrimaryKeyFlag means the column is primary key
    75  	PrimaryKeyFlag
    76  	// UniqueKeyFlag means the column is unique key
    77  	UniqueKeyFlag
    78  	// MultipleKeyFlag means the column is multiple key
    79  	MultipleKeyFlag
    80  	// NullableFlag means the column is nullable
    81  	NullableFlag
    82  	// UnsignedFlag means the column stores an unsigned integer
    83  	UnsignedFlag
    84  )
    85  
    86  // SetIsBinary sets BinaryFlag
    87  func (b *ColumnFlagType) SetIsBinary() {
    88  	(*util.Flag)(b).Add(util.Flag(BinaryFlag))
    89  }
    90  
    91  // UnsetIsBinary unsets BinaryFlag
    92  func (b *ColumnFlagType) UnsetIsBinary() {
    93  	(*util.Flag)(b).Remove(util.Flag(BinaryFlag))
    94  }
    95  
    96  // IsBinary shows whether BinaryFlag is set
    97  func (b *ColumnFlagType) IsBinary() bool {
    98  	return (*util.Flag)(b).HasAll(util.Flag(BinaryFlag))
    99  }
   100  
   101  // SetIsHandleKey sets HandleKey
   102  func (b *ColumnFlagType) SetIsHandleKey() {
   103  	(*util.Flag)(b).Add(util.Flag(HandleKeyFlag))
   104  }
   105  
   106  // UnsetIsHandleKey unsets HandleKey
   107  func (b *ColumnFlagType) UnsetIsHandleKey() {
   108  	(*util.Flag)(b).Remove(util.Flag(HandleKeyFlag))
   109  }
   110  
   111  // IsHandleKey shows whether HandleKey is set
   112  func (b *ColumnFlagType) IsHandleKey() bool {
   113  	return (*util.Flag)(b).HasAll(util.Flag(HandleKeyFlag))
   114  }
   115  
   116  // SetIsGeneratedColumn sets GeneratedColumn
   117  func (b *ColumnFlagType) SetIsGeneratedColumn() {
   118  	(*util.Flag)(b).Add(util.Flag(GeneratedColumnFlag))
   119  }
   120  
   121  // UnsetIsGeneratedColumn unsets GeneratedColumn
   122  func (b *ColumnFlagType) UnsetIsGeneratedColumn() {
   123  	(*util.Flag)(b).Remove(util.Flag(GeneratedColumnFlag))
   124  }
   125  
   126  // IsGeneratedColumn shows whether GeneratedColumn is set
   127  func (b *ColumnFlagType) IsGeneratedColumn() bool {
   128  	return (*util.Flag)(b).HasAll(util.Flag(GeneratedColumnFlag))
   129  }
   130  
   131  // SetIsPrimaryKey sets PrimaryKeyFlag
   132  func (b *ColumnFlagType) SetIsPrimaryKey() {
   133  	(*util.Flag)(b).Add(util.Flag(PrimaryKeyFlag))
   134  }
   135  
   136  // UnsetIsPrimaryKey unsets PrimaryKeyFlag
   137  func (b *ColumnFlagType) UnsetIsPrimaryKey() {
   138  	(*util.Flag)(b).Remove(util.Flag(PrimaryKeyFlag))
   139  }
   140  
   141  // IsPrimaryKey shows whether PrimaryKeyFlag is set
   142  func (b *ColumnFlagType) IsPrimaryKey() bool {
   143  	return (*util.Flag)(b).HasAll(util.Flag(PrimaryKeyFlag))
   144  }
   145  
   146  // SetIsUniqueKey sets UniqueKeyFlag
   147  func (b *ColumnFlagType) SetIsUniqueKey() {
   148  	(*util.Flag)(b).Add(util.Flag(UniqueKeyFlag))
   149  }
   150  
   151  // UnsetIsUniqueKey unsets UniqueKeyFlag
   152  func (b *ColumnFlagType) UnsetIsUniqueKey() {
   153  	(*util.Flag)(b).Remove(util.Flag(UniqueKeyFlag))
   154  }
   155  
   156  // IsUniqueKey shows whether UniqueKeyFlag is set
   157  func (b *ColumnFlagType) IsUniqueKey() bool {
   158  	return (*util.Flag)(b).HasAll(util.Flag(UniqueKeyFlag))
   159  }
   160  
   161  // IsMultipleKey shows whether MultipleKeyFlag is set
   162  func (b *ColumnFlagType) IsMultipleKey() bool {
   163  	return (*util.Flag)(b).HasAll(util.Flag(MultipleKeyFlag))
   164  }
   165  
   166  // SetIsMultipleKey sets MultipleKeyFlag
   167  func (b *ColumnFlagType) SetIsMultipleKey() {
   168  	(*util.Flag)(b).Add(util.Flag(MultipleKeyFlag))
   169  }
   170  
   171  // UnsetIsMultipleKey unsets MultipleKeyFlag
   172  func (b *ColumnFlagType) UnsetIsMultipleKey() {
   173  	(*util.Flag)(b).Remove(util.Flag(MultipleKeyFlag))
   174  }
   175  
   176  // IsNullable shows whether NullableFlag is set
   177  func (b *ColumnFlagType) IsNullable() bool {
   178  	return (*util.Flag)(b).HasAll(util.Flag(NullableFlag))
   179  }
   180  
   181  // SetIsNullable sets NullableFlag
   182  func (b *ColumnFlagType) SetIsNullable() {
   183  	(*util.Flag)(b).Add(util.Flag(NullableFlag))
   184  }
   185  
   186  // UnsetIsNullable unsets NullableFlag
   187  func (b *ColumnFlagType) UnsetIsNullable() {
   188  	(*util.Flag)(b).Remove(util.Flag(NullableFlag))
   189  }
   190  
   191  // IsUnsigned shows whether UnsignedFlag is set
   192  func (b *ColumnFlagType) IsUnsigned() bool {
   193  	return (*util.Flag)(b).HasAll(util.Flag(UnsignedFlag))
   194  }
   195  
   196  // SetIsUnsigned sets UnsignedFlag
   197  func (b *ColumnFlagType) SetIsUnsigned() {
   198  	(*util.Flag)(b).Add(util.Flag(UnsignedFlag))
   199  }
   200  
   201  // UnsetIsUnsigned unsets UnsignedFlag
   202  func (b *ColumnFlagType) UnsetIsUnsigned() {
   203  	(*util.Flag)(b).Remove(util.Flag(UnsignedFlag))
   204  }
   205  
   206  // TableName represents name of a table, includes table name and schema name.
   207  type TableName struct {
   208  	Schema      string `toml:"db-name" msg:"db-name"`
   209  	Table       string `toml:"tbl-name" msg:"tbl-name"`
   210  	TableID     int64  `toml:"tbl-id" msg:"tbl-id"`
   211  	IsPartition bool   `toml:"is-partition" msg:"is-partition"`
   212  }
   213  
   214  // String implements fmt.Stringer interface.
   215  func (t TableName) String() string {
   216  	return fmt.Sprintf("%s.%s", t.Schema, t.Table)
   217  }
   218  
   219  // QuoteString returns quoted full table name
   220  func (t TableName) QuoteString() string {
   221  	return quotes.QuoteSchema(t.Schema, t.Table)
   222  }
   223  
   224  // GetSchema returns schema name.
   225  func (t *TableName) GetSchema() string {
   226  	return t.Schema
   227  }
   228  
   229  // GetTable returns table name.
   230  func (t *TableName) GetTable() string {
   231  	return t.Table
   232  }
   233  
   234  // GetTableID returns table ID.
   235  func (t *TableName) GetTableID() int64 {
   236  	return t.TableID
   237  }
   238  
   239  // RedoLogType is the type of log
   240  type RedoLogType int
   241  
   242  const (
   243  	// RedoLogTypeUnknown is unknown type of log
   244  	RedoLogTypeUnknown RedoLogType = iota
   245  	// RedoLogTypeRow is row type of log
   246  	RedoLogTypeRow
   247  	// RedoLogTypeDDL is ddl type of log
   248  	RedoLogTypeDDL
   249  )
   250  
   251  // RedoLog defines the persistent structure of redo log
   252  // since MsgPack do not support types that are defined in another package,
   253  // more info https://github.com/tinylib/msgp/issues/158, https://github.com/tinylib/msgp/issues/149
   254  // so define a RedoColumn, RedoDDLEvent instead of using the Column, DDLEvent
   255  type RedoLog struct {
   256  	RedoRow RedoRowChangedEvent `msg:"row"`
   257  	RedoDDL RedoDDLEvent        `msg:"ddl"`
   258  	Type    RedoLogType         `msg:"type"`
   259  }
   260  
   261  // GetCommitTs returns the commit ts of the redo log.
   262  func (r *RedoLog) GetCommitTs() Ts {
   263  	switch r.Type {
   264  	case RedoLogTypeRow:
   265  		return r.RedoRow.Row.CommitTs
   266  	case RedoLogTypeDDL:
   267  		return r.RedoDDL.DDL.CommitTs
   268  	default:
   269  		log.Panic("invalid redo log type", zap.Any("type", r.Type))
   270  	}
   271  	return 0
   272  }
   273  
   274  // TrySplitAndSortUpdateEvent redo log do nothing
   275  func (r *RedoLog) TrySplitAndSortUpdateEvent(_ string) error {
   276  	return nil
   277  }
   278  
   279  // RedoRowChangedEvent represents the DML event used in RedoLog
   280  type RedoRowChangedEvent struct {
   281  	Row        *RowChangedEventInRedoLog `msg:"row"`
   282  	Columns    []RedoColumn              `msg:"columns"`
   283  	PreColumns []RedoColumn              `msg:"pre-columns"`
   284  }
   285  
   286  // RedoDDLEvent represents DDL event used in redo log persistent
   287  type RedoDDLEvent struct {
   288  	DDL       *DDLEvent `msg:"ddl"`
   289  	Type      byte      `msg:"type"`
   290  	TableName TableName `msg:"table-name"`
   291  }
   292  
   293  // ToRedoLog converts row changed event to redo log
   294  func (r *RowChangedEvent) ToRedoLog() *RedoLog {
   295  	rowInRedoLog := &RowChangedEventInRedoLog{
   296  		StartTs:  r.StartTs,
   297  		CommitTs: r.CommitTs,
   298  		Table: &TableName{
   299  			Schema:      r.TableInfo.GetSchemaName(),
   300  			Table:       r.TableInfo.GetTableName(),
   301  			TableID:     r.PhysicalTableID,
   302  			IsPartition: r.TableInfo.IsPartitionTable(),
   303  		},
   304  		Columns:      r.GetColumns(),
   305  		PreColumns:   r.GetPreColumns(),
   306  		IndexColumns: r.TableInfo.IndexColumnsOffset,
   307  	}
   308  	return &RedoLog{
   309  		RedoRow: RedoRowChangedEvent{Row: rowInRedoLog},
   310  		Type:    RedoLogTypeRow,
   311  	}
   312  }
   313  
   314  // ToRedoLog converts ddl event to redo log
   315  func (d *DDLEvent) ToRedoLog() *RedoLog {
   316  	return &RedoLog{
   317  		RedoDDL: RedoDDLEvent{DDL: d},
   318  		Type:    RedoLogTypeDDL,
   319  	}
   320  }
   321  
   322  // RowChangedEvent represents a row changed event
   323  //
   324  //msgp:ignore RowChangedEvent
   325  type RowChangedEvent struct {
   326  	StartTs  uint64
   327  	CommitTs uint64
   328  
   329  	RowID int64 // Deprecated. It is empty when the RowID comes from clustered index table.
   330  
   331  	PhysicalTableID int64
   332  
   333  	// NOTICE: We probably store the logical ID inside TableInfo's TableName,
   334  	// not the physical ID.
   335  	// For normal table, there is only one ID, which is the physical ID.
   336  	// AKA TIDB_TABLE_ID.
   337  	// For partitioned table, there are two kinds of ID:
   338  	// 1. TIDB_PARTITION_ID is the physical ID of the partition.
   339  	// 2. TIDB_TABLE_ID is the logical ID of the table.
   340  	// In general, we always use the physical ID to represent a table, but we
   341  	// record the logical ID from the DDL event(job.BinlogInfo.TableInfo).
   342  	// So be careful when using the TableInfo.
   343  	TableInfo *TableInfo
   344  
   345  	Columns    []*ColumnData
   346  	PreColumns []*ColumnData
   347  
   348  	// Checksum for the event, only not nil if the upstream TiDB enable the row level checksum
   349  	// and TiCDC set the integrity check level to the correctness.
   350  	Checksum *integrity.Checksum
   351  
   352  	// ApproximateDataSize is the approximate size of protobuf binary
   353  	// representation of this event.
   354  	ApproximateDataSize int64
   355  
   356  	// SplitTxn marks this RowChangedEvent as the first line of a new txn.
   357  	SplitTxn bool
   358  	// ReplicatingTs is ts when a table starts replicating events to downstream.
   359  	ReplicatingTs Ts
   360  	// HandleKey is the key of the row changed event.
   361  	// It can be used to identify the row changed event.
   362  	// It can be one of three : common_handle, int_handle or _tidb_rowid based on the table definitions
   363  	// 1. primary key is the clustered index, and key is not int type, then we use `CommonHandle`
   364  	// 2. primary key is int type(including different types of int, such as bigint, TINYINT), then we use IntHandle
   365  	// 3. when the table doesn't have the primary key and clustered index,
   366  	//    tidb will make a hidden column called "_tidb_rowid" as the handle.
   367  	//    due to the type of "_tidb_rowid" is int, so we also use IntHandle to represent.
   368  	HandleKey kv.Handle
   369  }
   370  
   371  // RowChangedEventInRedoLog is used to store RowChangedEvent in redo log v2 format
   372  type RowChangedEventInRedoLog struct {
   373  	StartTs  uint64 `msg:"start-ts"`
   374  	CommitTs uint64 `msg:"commit-ts"`
   375  
   376  	// Table contains the table name and table ID.
   377  	// NOTICE: We store the physical table ID here, not the logical table ID.
   378  	Table *TableName `msg:"table"`
   379  
   380  	Columns      []*Column `msg:"columns"`
   381  	PreColumns   []*Column `msg:"pre-columns"`
   382  	IndexColumns [][]int   `msg:"index-columns"`
   383  }
   384  
   385  // ToRowChangedEvent converts RowChangedEventInRedoLog to RowChangedEvent
   386  func (r *RowChangedEventInRedoLog) ToRowChangedEvent() *RowChangedEvent {
   387  	cols := r.Columns
   388  	if cols == nil {
   389  		cols = r.PreColumns
   390  	}
   391  	tableInfo := BuildTableInfo(
   392  		r.Table.Schema,
   393  		r.Table.Table,
   394  		cols,
   395  		r.IndexColumns)
   396  	tableInfo.TableName.TableID = r.Table.TableID
   397  	tableInfo.TableName.IsPartition = r.Table.IsPartition
   398  	row := &RowChangedEvent{
   399  		StartTs:         r.StartTs,
   400  		CommitTs:        r.CommitTs,
   401  		PhysicalTableID: r.Table.TableID,
   402  		TableInfo:       tableInfo,
   403  		Columns:         Columns2ColumnDatas(r.Columns, tableInfo),
   404  		PreColumns:      Columns2ColumnDatas(r.PreColumns, tableInfo),
   405  	}
   406  	return row
   407  }
   408  
   409  // txnRows represents a set of events that belong to the same transaction.
   410  type txnRows []*RowChangedEvent
   411  
   412  // Len is the number of elements in the collection.
   413  func (e txnRows) Len() int {
   414  	return len(e)
   415  }
   416  
   417  // Less sort the events base on the order of event type delete<update<insert
   418  func (e txnRows) Less(i, j int) bool {
   419  	return getDMLOrder(e[i]) < getDMLOrder(e[j])
   420  }
   421  
   422  // getDMLOrder returns the order of the dml types: delete<update<insert
   423  func getDMLOrder(event *RowChangedEvent) int {
   424  	if event.IsDelete() {
   425  		return typeDelete
   426  	} else if event.IsUpdate() {
   427  		return typeUpdate
   428  	}
   429  	return typeInsert
   430  }
   431  
   432  func (e txnRows) Swap(i, j int) {
   433  	e[i], e[j] = e[j], e[i]
   434  }
   435  
   436  // GetCommitTs returns the commit timestamp of this event.
   437  func (r *RowChangedEvent) GetCommitTs() uint64 {
   438  	return r.CommitTs
   439  }
   440  
   441  // TrySplitAndSortUpdateEvent do nothing
   442  func (r *RowChangedEvent) TrySplitAndSortUpdateEvent(_ string) error {
   443  	return nil
   444  }
   445  
   446  // IsDelete returns true if the row is a delete event
   447  func (r *RowChangedEvent) IsDelete() bool {
   448  	return len(r.PreColumns) != 0 && len(r.Columns) == 0
   449  }
   450  
   451  // IsInsert returns true if the row is an insert event
   452  func (r *RowChangedEvent) IsInsert() bool {
   453  	return len(r.PreColumns) == 0 && len(r.Columns) != 0
   454  }
   455  
   456  // IsUpdate returns true if the row is an update event
   457  func (r *RowChangedEvent) IsUpdate() bool {
   458  	return len(r.PreColumns) != 0 && len(r.Columns) != 0
   459  }
   460  
   461  func columnData2Column(col *ColumnData, tableInfo *TableInfo) *Column {
   462  	colID := col.ColumnID
   463  	offset, ok := tableInfo.columnsOffset[colID]
   464  	if !ok {
   465  		log.Panic("invalid column id",
   466  			zap.Int64("columnID", colID),
   467  			zap.Any("tableInfo", tableInfo))
   468  	}
   469  	colInfo := tableInfo.Columns[offset]
   470  	return &Column{
   471  		Name:      colInfo.Name.O,
   472  		Type:      colInfo.GetType(),
   473  		Charset:   colInfo.GetCharset(),
   474  		Collation: colInfo.GetCollate(),
   475  		Flag:      *tableInfo.ColumnsFlag[colID],
   476  		Value:     col.Value,
   477  		Default:   GetColumnDefaultValue(colInfo),
   478  	}
   479  }
   480  
   481  func columnDatas2Columns(cols []*ColumnData, tableInfo *TableInfo) []*Column {
   482  	if cols == nil {
   483  		return nil
   484  	}
   485  	columns := make([]*Column, len(cols))
   486  	for i, colData := range cols {
   487  		if colData == nil {
   488  			log.Warn("meet nil column data, should not happened in production env",
   489  				zap.Any("cols", cols),
   490  				zap.Any("tableInfo", tableInfo))
   491  			continue
   492  		}
   493  		columns[i] = columnData2Column(colData, tableInfo)
   494  	}
   495  	return columns
   496  }
   497  
   498  // GetColumns returns the columns of the event
   499  func (r *RowChangedEvent) GetColumns() []*Column {
   500  	return columnDatas2Columns(r.Columns, r.TableInfo)
   501  }
   502  
   503  // GetPreColumns returns the pre columns of the event
   504  func (r *RowChangedEvent) GetPreColumns() []*Column {
   505  	return columnDatas2Columns(r.PreColumns, r.TableInfo)
   506  }
   507  
   508  // PrimaryKeyColumnNames return all primary key's name
   509  func (r *RowChangedEvent) PrimaryKeyColumnNames() []string {
   510  	var result []string
   511  
   512  	var cols []*ColumnData
   513  	if r.IsDelete() {
   514  		cols = r.PreColumns
   515  	} else {
   516  		cols = r.Columns
   517  	}
   518  
   519  	result = make([]string, 0)
   520  	tableInfo := r.TableInfo
   521  	for _, col := range cols {
   522  		if col != nil && tableInfo.ForceGetColumnFlagType(col.ColumnID).IsPrimaryKey() {
   523  			result = append(result, tableInfo.ForceGetColumnName(col.ColumnID))
   524  		}
   525  	}
   526  	return result
   527  }
   528  
   529  // GetHandleKeyColumnValues returns all handle key's column values
   530  func (r *RowChangedEvent) GetHandleKeyColumnValues() []string {
   531  	var result []string
   532  
   533  	var cols []*ColumnData
   534  	if r.IsDelete() {
   535  		cols = r.PreColumns
   536  	} else {
   537  		cols = r.Columns
   538  	}
   539  
   540  	result = make([]string, 0)
   541  	tableInfo := r.TableInfo
   542  	for _, col := range cols {
   543  		if col != nil && tableInfo.ForceGetColumnFlagType(col.ColumnID).IsHandleKey() {
   544  			result = append(result, ColumnValueString(col.Value))
   545  		}
   546  	}
   547  	return result
   548  }
   549  
   550  // HandleKeyColInfos returns the column(s) and colInfo(s) corresponding to the handle key(s)
   551  func (r *RowChangedEvent) HandleKeyColInfos() ([]*Column, []rowcodec.ColInfo) {
   552  	pkeyCols := make([]*Column, 0)
   553  	pkeyColInfos := make([]rowcodec.ColInfo, 0)
   554  
   555  	var cols []*ColumnData
   556  	if r.IsDelete() {
   557  		cols = r.PreColumns
   558  	} else {
   559  		cols = r.Columns
   560  	}
   561  
   562  	tableInfo := r.TableInfo
   563  	colInfos := tableInfo.GetColInfosForRowChangedEvent()
   564  	for i, col := range cols {
   565  		if col != nil && tableInfo.ForceGetColumnFlagType(col.ColumnID).IsHandleKey() {
   566  			pkeyCols = append(pkeyCols, columnData2Column(col, tableInfo))
   567  			pkeyColInfos = append(pkeyColInfos, colInfos[i])
   568  		}
   569  	}
   570  
   571  	// It is okay not to have handle keys, so the empty array is an acceptable result
   572  	return pkeyCols, pkeyColInfos
   573  }
   574  
   575  // ApproximateBytes returns approximate bytes in memory consumed by the event.
   576  func (r *RowChangedEvent) ApproximateBytes() int {
   577  	const sizeOfRowEvent = int(unsafe.Sizeof(*r))
   578  
   579  	size := 0
   580  	// Size of cols
   581  	for i := range r.Columns {
   582  		size += r.Columns[i].ApproximateBytes
   583  	}
   584  	// Size of pre cols
   585  	for i := range r.PreColumns {
   586  		if r.PreColumns[i] != nil {
   587  			size += r.PreColumns[i].ApproximateBytes
   588  		}
   589  	}
   590  	// Size of an empty row event
   591  	size += sizeOfRowEvent
   592  	return size
   593  }
   594  
   595  // Columns2ColumnDatas convert `Column`s to `ColumnData`s
   596  func Columns2ColumnDatas(cols []*Column, tableInfo *TableInfo) []*ColumnData {
   597  	if cols == nil {
   598  		return nil
   599  	}
   600  	columns := make([]*ColumnData, len(cols))
   601  	for i, col := range cols {
   602  		if col == nil {
   603  			continue
   604  		}
   605  		colID := tableInfo.ForceGetColumnIDByName(col.Name)
   606  		columns[i] = &ColumnData{
   607  			ColumnID: colID,
   608  			Value:    col.Value,
   609  		}
   610  	}
   611  	return columns
   612  }
   613  
   614  // Column represents a column value and its schema info
   615  type Column struct {
   616  	Name      string         `msg:"name"`
   617  	Type      byte           `msg:"type"`
   618  	Charset   string         `msg:"charset"`
   619  	Collation string         `msg:"collation"`
   620  	Flag      ColumnFlagType `msg:"-"`
   621  	Value     interface{}    `msg:"-"`
   622  	Default   interface{}    `msg:"-"`
   623  
   624  	// ApproximateBytes is approximate bytes consumed by the column.
   625  	ApproximateBytes int `msg:"-"`
   626  }
   627  
   628  // ColumnData represents a column value in row changed event
   629  type ColumnData struct {
   630  	// ColumnID may be just a mock id, because we don't store it in redo log.
   631  	// So after restore from redo log, we need to give every a column a mock id.
   632  	// The only guarantee is that the column id is unique in a RowChangedEvent
   633  	ColumnID int64       `json:"column_id" msg:"column_id"`
   634  	Value    interface{} `json:"value" msg:"-"`
   635  
   636  	// ApproximateBytes is approximate bytes consumed by the column.
   637  	ApproximateBytes int `json:"-" msg:"-"`
   638  }
   639  
   640  // RedoColumn stores Column change
   641  type RedoColumn struct {
   642  	// Fields from Column and can't be marshaled directly in Column.
   643  	Value interface{} `msg:"column"`
   644  	// msgp transforms empty byte slice into nil, PTAL msgp#247.
   645  	ValueIsEmptyBytes bool   `msg:"value-is-empty-bytes"`
   646  	Flag              uint64 `msg:"flag"`
   647  }
   648  
   649  // ColumnIDAllocator represents the interface to allocate column id for tableInfo
   650  type ColumnIDAllocator interface {
   651  	// GetColumnID return the column id according to the column name
   652  	GetColumnID(name string) int64
   653  }
   654  
   655  // IncrementalColumnIDAllocator allocates column id in an incremental way.
   656  // At most of the time, it is the default implementation when you don't care the column id's concrete value.
   657  //
   658  //msgp:ignore IncrementalColumnIDAllocator
   659  type IncrementalColumnIDAllocator struct {
   660  	nextColID int64
   661  }
   662  
   663  // NewIncrementalColumnIDAllocator creates a new IncrementalColumnIDAllocator
   664  func NewIncrementalColumnIDAllocator() *IncrementalColumnIDAllocator {
   665  	return &IncrementalColumnIDAllocator{
   666  		nextColID: 100, // 100 is an arbitrary number
   667  	}
   668  }
   669  
   670  // GetColumnID return the next mock column id
   671  func (d *IncrementalColumnIDAllocator) GetColumnID(name string) int64 {
   672  	result := d.nextColID
   673  	d.nextColID += 1
   674  	return result
   675  }
   676  
   677  // NameBasedColumnIDAllocator allocates column id using an prefined map from column name to id
   678  //
   679  //msgp:ignore NameBasedColumnIDAllocator
   680  type NameBasedColumnIDAllocator struct {
   681  	nameToIDMap map[string]int64
   682  }
   683  
   684  // NewNameBasedColumnIDAllocator creates a new NameBasedColumnIDAllocator
   685  func NewNameBasedColumnIDAllocator(nameToIDMap map[string]int64) *NameBasedColumnIDAllocator {
   686  	return &NameBasedColumnIDAllocator{
   687  		nameToIDMap: nameToIDMap,
   688  	}
   689  }
   690  
   691  // GetColumnID return the column id of the name
   692  func (n *NameBasedColumnIDAllocator) GetColumnID(name string) int64 {
   693  	colID, ok := n.nameToIDMap[name]
   694  	if !ok {
   695  		log.Panic("column not found",
   696  			zap.String("name", name),
   697  			zap.Any("nameToIDMap", n.nameToIDMap))
   698  	}
   699  	return colID
   700  }
   701  
   702  // BuildTableInfo builds a table info from given information.
   703  // Note that some fields of the result TableInfo may just be mocked.
   704  // The only guarantee is that we can use the result to reconstrut the information in `Column`.
   705  // The main use cases of this function it to build TableInfo from redo log and in tests.
   706  func BuildTableInfo(schemaName, tableName string, columns []*Column, indexColumns [][]int) *TableInfo {
   707  	tidbTableInfo := BuildTiDBTableInfo(tableName, columns, indexColumns)
   708  	return WrapTableInfo(100 /* not used */, schemaName, 1000 /* not used */, tidbTableInfo)
   709  }
   710  
   711  // BuildTableInfoWithPKNames4Test builds a table info from given information.
   712  func BuildTableInfoWithPKNames4Test(schemaName, tableName string, columns []*Column, pkNames map[string]struct{}) *TableInfo {
   713  	if len(pkNames) == 0 {
   714  		return BuildTableInfo(schemaName, tableName, columns, nil)
   715  	}
   716  	indexColumns := make([][]int, 1)
   717  	indexColumns[0] = make([]int, 0)
   718  	for i, col := range columns {
   719  		if _, ok := pkNames[col.Name]; ok {
   720  			indexColumns[0] = append(indexColumns[0], i)
   721  			col.Flag.SetIsHandleKey()
   722  			col.Flag.SetIsPrimaryKey()
   723  		}
   724  	}
   725  	if len(indexColumns[0]) != len(pkNames) {
   726  		log.Panic("cannot find all pks",
   727  			zap.Any("indexColumns", indexColumns),
   728  			zap.Any("pkNames", pkNames))
   729  	}
   730  	return BuildTableInfo(schemaName, tableName, columns, indexColumns)
   731  }
   732  
   733  // AddExtraColumnInfo is used to add some extra column info to the table info.
   734  // Just use it in test.
   735  func AddExtraColumnInfo(tableInfo *model.TableInfo, extraColInfos []rowcodec.ColInfo) {
   736  	for i, colInfo := range extraColInfos {
   737  		tableInfo.Columns[i].SetElems(colInfo.Ft.GetElems())
   738  		tableInfo.Columns[i].SetFlen(colInfo.Ft.GetFlen())
   739  	}
   740  }
   741  
   742  // GetHandleAndUniqueIndexOffsets4Test is used to get the offsets of handle columns and other unique index columns in test
   743  func GetHandleAndUniqueIndexOffsets4Test(cols []*Column) [][]int {
   744  	result := make([][]int, 0)
   745  	handleColumns := make([]int, 0)
   746  	for i, col := range cols {
   747  		if col.Flag.IsHandleKey() {
   748  			handleColumns = append(handleColumns, i)
   749  		} else if col.Flag.IsUniqueKey() {
   750  			// When there is a unique key which is not handle key,
   751  			// we cannot get the accurate index info for this key.
   752  			// So just be aggressive to make each unique column a unique index
   753  			// to make sure there is no write conflict when syncing data in tests.
   754  			result = append(result, []int{i})
   755  		}
   756  	}
   757  	if len(handleColumns) != 0 {
   758  		result = append(result, handleColumns)
   759  	}
   760  	return result
   761  }
   762  
   763  // BuildTiDBTableInfoWithoutVirtualColumns build a TableInfo without virual columns from the source table info
   764  func BuildTiDBTableInfoWithoutVirtualColumns(source *model.TableInfo) *model.TableInfo {
   765  	ret := source.Clone()
   766  	ret.Columns = make([]*model.ColumnInfo, 0, len(source.Columns))
   767  	rowColumnsCurrentOffset := 0
   768  	columnsOffset := make(map[string]int, len(source.Columns))
   769  	for _, srcCol := range source.Columns {
   770  		if !IsColCDCVisible(srcCol) {
   771  			continue
   772  		}
   773  		colInfo := srcCol.Clone()
   774  		colInfo.Offset = rowColumnsCurrentOffset
   775  		ret.Columns = append(ret.Columns, colInfo)
   776  		columnsOffset[colInfo.Name.O] = rowColumnsCurrentOffset
   777  		rowColumnsCurrentOffset += 1
   778  	}
   779  	// Keep all the index info even if it contains virtual columns for simplicity
   780  	for _, indexInfo := range ret.Indices {
   781  		for _, col := range indexInfo.Columns {
   782  			col.Offset = columnsOffset[col.Name.O]
   783  		}
   784  	}
   785  
   786  	return ret
   787  }
   788  
   789  // BuildTiDBTableInfo is a simple wrapper over BuildTiDBTableInfoImpl which create a default ColumnIDAllocator.
   790  func BuildTiDBTableInfo(tableName string, columns []*Column, indexColumns [][]int) *model.TableInfo {
   791  	return BuildTiDBTableInfoImpl(tableName, columns, indexColumns, NewIncrementalColumnIDAllocator())
   792  }
   793  
   794  // BuildTiDBTableInfoImpl builds a TiDB TableInfo from given information.
   795  // Note the result TableInfo may not be same as the original TableInfo in tidb.
   796  // The only guarantee is that you can restore the `Name`, `Type`, `Charset`, `Collation`
   797  // and `Flag` field of `Column` using the result TableInfo.
   798  // The precondition required for calling this function:
   799  //  1. There must be at least one handle key in `columns`;
   800  //  2. The handle key must either be a primary key or a non null unique key;
   801  //  3. The index that is selected as the handle must be provided in `indexColumns`;
   802  func BuildTiDBTableInfoImpl(
   803  	tableName string,
   804  	columns []*Column,
   805  	indexColumns [][]int,
   806  	columnIDAllocator ColumnIDAllocator,
   807  ) *model.TableInfo {
   808  	ret := &model.TableInfo{}
   809  	ret.Name = model.NewCIStr(tableName)
   810  
   811  	hasPrimaryKeyColumn := false
   812  	for i, col := range columns {
   813  		columnInfo := &model.ColumnInfo{
   814  			Offset: i,
   815  			State:  model.StatePublic,
   816  		}
   817  		if col == nil {
   818  			// actually, col should never be nil according to `datum2Column` and `WrapTableInfo` in prod env
   819  			// we mock it as generated column just for test
   820  			columnInfo.Name = model.NewCIStr("omitted")
   821  			columnInfo.GeneratedExprString = "pass_generated_check"
   822  			columnInfo.GeneratedStored = false
   823  			ret.Columns = append(ret.Columns, columnInfo)
   824  			continue
   825  		}
   826  		// add a mock id to identify columns inside cdc
   827  		columnInfo.ID = columnIDAllocator.GetColumnID(col.Name)
   828  		columnInfo.Name = model.NewCIStr(col.Name)
   829  		columnInfo.SetType(col.Type)
   830  
   831  		if col.Collation != "" {
   832  			columnInfo.SetCollate(col.Collation)
   833  		} else {
   834  			// collation is not stored, give it a default value
   835  			columnInfo.SetCollate(mysql.UTF8MB4DefaultCollation)
   836  		}
   837  
   838  		// inverse initColumnsFlag
   839  		flag := col.Flag
   840  		if col.Charset != "" {
   841  			columnInfo.SetCharset(col.Charset)
   842  		} else if flag.IsBinary() {
   843  			columnInfo.SetCharset("binary")
   844  		} else {
   845  			// charset is not stored, give it a default value
   846  			columnInfo.SetCharset(mysql.UTF8MB4Charset)
   847  		}
   848  		if flag.IsGeneratedColumn() {
   849  			// we do not use this field, so we set it to any non-empty string
   850  			columnInfo.GeneratedExprString = "pass_generated_check"
   851  			columnInfo.GeneratedStored = true
   852  		}
   853  		if flag.IsPrimaryKey() {
   854  			columnInfo.AddFlag(mysql.PriKeyFlag)
   855  			hasPrimaryKeyColumn = true
   856  			if !flag.IsHandleKey() {
   857  				log.Panic("Primary key must be handle key",
   858  					zap.String("table", tableName),
   859  					zap.Any("columns", columns),
   860  					zap.Any("indexColumns", indexColumns))
   861  			}
   862  			// just set it for test compatibility,
   863  			// actually we cannot deduce the value of IsCommonHandle from the provided args.
   864  			ret.IsCommonHandle = true
   865  		}
   866  		if flag.IsUniqueKey() {
   867  			columnInfo.AddFlag(mysql.UniqueKeyFlag)
   868  		}
   869  		if flag.IsHandleKey() {
   870  			if !flag.IsPrimaryKey() && !flag.IsUniqueKey() {
   871  				log.Panic("Handle key must either be primary key or unique key",
   872  					zap.String("table", tableName),
   873  					zap.Any("columns", columns),
   874  					zap.Any("indexColumns", indexColumns))
   875  			}
   876  		}
   877  		if !flag.IsNullable() {
   878  			columnInfo.AddFlag(mysql.NotNullFlag)
   879  		}
   880  		if flag.IsMultipleKey() {
   881  			columnInfo.AddFlag(mysql.MultipleKeyFlag)
   882  		}
   883  		if flag.IsUnsigned() {
   884  			columnInfo.AddFlag(mysql.UnsignedFlag)
   885  		}
   886  		ret.Columns = append(ret.Columns, columnInfo)
   887  	}
   888  
   889  	hasPrimaryKeyIndex := false
   890  	hasHandleIndex := false
   891  	// TiCDC handles columns according to the following rules:
   892  	// 1. If a primary key (PK) exists, it is chosen.
   893  	// 2. If there is no PK, TiCDC looks for a not null unique key (UK) with the least number of columns and the smallest index ID.
   894  	// So we assign the smallest index id to the index which is selected as handle to mock this behavior.
   895  	minIndexID := int64(1)
   896  	nextMockIndexID := minIndexID + 1
   897  	for i, colOffsets := range indexColumns {
   898  		indexInfo := &model.IndexInfo{
   899  			Name:  model.NewCIStr(fmt.Sprintf("idx_%d", i)),
   900  			State: model.StatePublic,
   901  		}
   902  		firstCol := columns[colOffsets[0]]
   903  		if firstCol == nil {
   904  			// when the referenced column is nil, we already have a handle index
   905  			// so we can skip this index.
   906  			// only happens for DELETE event and old value feature is disabled
   907  			continue
   908  		}
   909  		if firstCol.Flag.IsPrimaryKey() {
   910  			indexInfo.Unique = true
   911  		}
   912  		if firstCol.Flag.IsUniqueKey() {
   913  			indexInfo.Unique = true
   914  		}
   915  
   916  		isPrimary := true
   917  		isAllColumnsHandle := true
   918  		for _, offset := range colOffsets {
   919  			col := columns[offset]
   920  			// When only all columns in the index are primary key, then the index is primary key.
   921  			if col == nil || !col.Flag.IsPrimaryKey() {
   922  				isPrimary = false
   923  			}
   924  			if col == nil || !col.Flag.IsHandleKey() {
   925  				isAllColumnsHandle = false
   926  			}
   927  
   928  			tiCol := ret.Columns[offset]
   929  			indexCol := &model.IndexColumn{}
   930  			indexCol.Name = tiCol.Name
   931  			indexCol.Offset = offset
   932  			indexInfo.Columns = append(indexInfo.Columns, indexCol)
   933  			indexInfo.Primary = isPrimary
   934  		}
   935  		hasPrimaryKeyIndex = hasPrimaryKeyIndex || isPrimary
   936  		if isAllColumnsHandle {
   937  			// If there is no primary index, only one index will contain columns which are all handles.
   938  			// If there is a primary index, the primary index must be the handle.
   939  			// And there may be another index which is a subset of the primary index. So we skip this check.
   940  			if hasHandleIndex && !hasPrimaryKeyColumn {
   941  				log.Panic("Multiple handle index found",
   942  					zap.String("table", tableName),
   943  					zap.Any("colOffsets", colOffsets),
   944  					zap.String("indexName", indexInfo.Name.O),
   945  					zap.Any("columns", columns),
   946  					zap.Any("indexColumns", indexColumns))
   947  			}
   948  			hasHandleIndex = true
   949  		}
   950  		// If there is no primary column, we need allocate the min index id to the one selected as handle.
   951  		// In other cases, we don't care the concrete value of index id.
   952  		if isAllColumnsHandle && !hasPrimaryKeyColumn {
   953  			indexInfo.ID = minIndexID
   954  		} else {
   955  			indexInfo.ID = nextMockIndexID
   956  			nextMockIndexID += 1
   957  		}
   958  
   959  		// TODO: revert the "all column set index related flag" to "only the
   960  		// first column set index related flag" if needed
   961  
   962  		ret.Indices = append(ret.Indices, indexInfo)
   963  	}
   964  	if hasPrimaryKeyColumn != hasPrimaryKeyIndex {
   965  		log.Panic("Primary key column and primary key index is not consistent",
   966  			zap.String("table", tableName),
   967  			zap.Any("columns", columns),
   968  			zap.Any("indexColumns", indexColumns),
   969  			zap.Bool("hasPrimaryKeyColumn", hasPrimaryKeyColumn),
   970  			zap.Bool("hasPrimaryKeyIndex", hasPrimaryKeyIndex))
   971  	}
   972  	return ret
   973  }
   974  
   975  // ColumnValueString returns the string representation of the column value
   976  func ColumnValueString(c interface{}) string {
   977  	var data string
   978  	switch v := c.(type) {
   979  	case nil:
   980  		data = "null"
   981  	case bool:
   982  		if v {
   983  			data = "1"
   984  		} else {
   985  			data = "0"
   986  		}
   987  	case int:
   988  		data = strconv.FormatInt(int64(v), 10)
   989  	case int8:
   990  		data = strconv.FormatInt(int64(v), 10)
   991  	case int16:
   992  		data = strconv.FormatInt(int64(v), 10)
   993  	case int32:
   994  		data = strconv.FormatInt(int64(v), 10)
   995  	case int64:
   996  		data = strconv.FormatInt(v, 10)
   997  	case uint8:
   998  		data = strconv.FormatUint(uint64(v), 10)
   999  	case uint16:
  1000  		data = strconv.FormatUint(uint64(v), 10)
  1001  	case uint32:
  1002  		data = strconv.FormatUint(uint64(v), 10)
  1003  	case uint64:
  1004  		data = strconv.FormatUint(v, 10)
  1005  	case float32:
  1006  		data = strconv.FormatFloat(float64(v), 'f', -1, 32)
  1007  	case float64:
  1008  		data = strconv.FormatFloat(v, 'f', -1, 64)
  1009  	case string:
  1010  		data = v
  1011  	case []byte:
  1012  		data = string(v)
  1013  	default:
  1014  		data = fmt.Sprintf("%v", v)
  1015  	}
  1016  	return data
  1017  }
  1018  
  1019  // DDLEvent stores DDL event
  1020  type DDLEvent struct {
  1021  	StartTs      uint64           `msg:"start-ts"`
  1022  	CommitTs     uint64           `msg:"commit-ts"`
  1023  	Query        string           `msg:"query"`
  1024  	TableInfo    *TableInfo       `msg:"-"`
  1025  	PreTableInfo *TableInfo       `msg:"-"`
  1026  	Type         model.ActionType `msg:"-"`
  1027  	Done         atomic.Bool      `msg:"-"`
  1028  	Charset      string           `msg:"-"`
  1029  	Collate      string           `msg:"-"`
  1030  	IsBootstrap  bool             `msg:"-"`
  1031  	// BDRRole is the role of the TiDB cluster, it is used to determine whether
  1032  	// the DDL is executed by the primary cluster.
  1033  	BDRRole string        `msg:"-"`
  1034  	SQLMode mysql.SQLMode `msg:"-"`
  1035  }
  1036  
  1037  // FromJob fills the values with DDLEvent from DDL job
  1038  func (d *DDLEvent) FromJob(job *model.Job, preTableInfo *TableInfo, tableInfo *TableInfo) {
  1039  	d.FromJobWithArgs(job, preTableInfo, tableInfo, "", "")
  1040  }
  1041  
  1042  // FromJobWithArgs fills the values with DDLEvent from DDL job
  1043  func (d *DDLEvent) FromJobWithArgs(
  1044  	job *model.Job,
  1045  	preTableInfo, tableInfo *TableInfo,
  1046  	oldSchemaName, newSchemaName string,
  1047  ) {
  1048  	d.StartTs = job.StartTS
  1049  	d.CommitTs = job.BinlogInfo.FinishedTS
  1050  	d.Type = job.Type
  1051  	d.PreTableInfo = preTableInfo
  1052  	d.TableInfo = tableInfo
  1053  	d.Charset = job.Charset
  1054  	d.Collate = job.Collate
  1055  	d.BDRRole = job.BDRRole
  1056  	d.SQLMode = job.SQLMode
  1057  	switch d.Type {
  1058  	// The query for "DROP TABLE" and "DROP VIEW" statements need
  1059  	// to be rebuilt. The reason is elaborated as follows:
  1060  	// for a DDL statement like "DROP TABLE test1.table1, test2.table2",
  1061  	// two DDL jobs will be generated. These two jobs can be differentiated
  1062  	// from job.BinlogInfo.TableInfo whereas the job.Query are identical.
  1063  	case model.ActionDropTable:
  1064  		d.Query = fmt.Sprintf("DROP TABLE `%s`.`%s`",
  1065  			d.TableInfo.TableName.Schema, d.TableInfo.TableName.Table)
  1066  	case model.ActionDropView:
  1067  		d.Query = fmt.Sprintf("DROP VIEW `%s`.`%s`",
  1068  			d.TableInfo.TableName.Schema, d.TableInfo.TableName.Table)
  1069  	case model.ActionRenameTables:
  1070  		oldTableName := preTableInfo.Name.O
  1071  		newTableName := tableInfo.Name.O
  1072  		d.Query = fmt.Sprintf("RENAME TABLE `%s`.`%s` TO `%s`.`%s`",
  1073  			oldSchemaName, oldTableName, newSchemaName, newTableName)
  1074  		// Note that type is ActionRenameTable, not ActionRenameTables.
  1075  		d.Type = model.ActionRenameTable
  1076  	case model.ActionExchangeTablePartition:
  1077  		// Parse idx of partition name from query.
  1078  		upperQuery := strings.ToUpper(job.Query)
  1079  		idx1 := strings.Index(upperQuery, "EXCHANGE PARTITION") + len("EXCHANGE PARTITION")
  1080  		idx2 := strings.Index(upperQuery, "WITH TABLE")
  1081  
  1082  		// Note that partition name should be parsed from original query, not the upperQuery.
  1083  		partName := strings.TrimSpace(job.Query[idx1:idx2])
  1084  		// The tableInfo is the partition table, preTableInfo is non partition table.
  1085  		d.Query = fmt.Sprintf("ALTER TABLE `%s`.`%s` EXCHANGE PARTITION `%s` WITH TABLE `%s`.`%s`",
  1086  			tableInfo.TableName.Schema, tableInfo.TableName.Table, partName,
  1087  			preTableInfo.TableName.Schema, preTableInfo.TableName.Table)
  1088  
  1089  		if strings.HasSuffix(upperQuery, "WITHOUT VALIDATION") {
  1090  			d.Query += " WITHOUT VALIDATION"
  1091  		}
  1092  	default:
  1093  		d.Query = job.Query
  1094  	}
  1095  }
  1096  
  1097  // NewBootstrapDDLEvent returns a bootstrap DDL event.
  1098  // We set Bootstrap DDL event's startTs and commitTs to 0.
  1099  // Because it is generated by the TiCDC, not from the upstream TiDB.
  1100  // And they ere useless for a bootstrap DDL event.
  1101  func NewBootstrapDDLEvent(tableInfo *TableInfo) *DDLEvent {
  1102  	return &DDLEvent{
  1103  		StartTs:     0,
  1104  		CommitTs:    0,
  1105  		TableInfo:   tableInfo,
  1106  		IsBootstrap: true,
  1107  	}
  1108  }
  1109  
  1110  // SingleTableTxn represents a transaction which includes many row events in a single table
  1111  //
  1112  //msgp:ignore SingleTableTxn
  1113  type SingleTableTxn struct {
  1114  	PhysicalTableID int64
  1115  	TableInfo       *TableInfo
  1116  	// TableInfoVersion is the version of the table info, it is used to generate data path
  1117  	// in storage sink. Generally, TableInfoVersion equals to `SingleTableTxn.TableInfo.Version`.
  1118  	// Besides, if one table is just scheduled to a new processor, the TableInfoVersion should be
  1119  	// greater than or equal to the startTs of table sink.
  1120  	TableInfoVersion uint64
  1121  
  1122  	StartTs  uint64
  1123  	CommitTs uint64
  1124  	Rows     []*RowChangedEvent
  1125  }
  1126  
  1127  // GetCommitTs returns the commit timestamp of the transaction.
  1128  func (t *SingleTableTxn) GetCommitTs() uint64 {
  1129  	return t.CommitTs
  1130  }
  1131  
  1132  // GetPhysicalTableID returns the physical table id of the table in the transaction
  1133  func (t *SingleTableTxn) GetPhysicalTableID() int64 {
  1134  	return t.PhysicalTableID
  1135  }
  1136  
  1137  // TrySplitAndSortUpdateEvent split update events if unique key is updated
  1138  func (t *SingleTableTxn) TrySplitAndSortUpdateEvent(scheme string) error {
  1139  	if !t.shouldSplitUpdateEvent(scheme) {
  1140  		return nil
  1141  	}
  1142  	newRows, err := trySplitAndSortUpdateEvent(t.Rows)
  1143  	if err != nil {
  1144  		return errors.Trace(err)
  1145  	}
  1146  	t.Rows = newRows
  1147  	return nil
  1148  }
  1149  
  1150  // Whether split a single update event into delete and insert events?
  1151  //
  1152  // For the MySQL Sink, we don't split any update event.
  1153  // This may cause error like "duplicate entry" when sink to the downstream.
  1154  // This kind of error will cause the changefeed to restart,
  1155  // and then the related update rows will be splitted to insert and delete at puller side.
  1156  //
  1157  // For the Kafka and Storage sink, always split a single unique key changed update event, since:
  1158  // 1. Avro and CSV does not output the previous column values for the update event, so it would
  1159  // cause consumer missing data if the unique key changed event is not split.
  1160  // 2. Index-Value Dispatcher cannot work correctly if the unique key changed event isn't split.
  1161  func (t *SingleTableTxn) shouldSplitUpdateEvent(sinkScheme string) bool {
  1162  	return !sink.IsMySQLCompatibleScheme(sinkScheme)
  1163  }
  1164  
  1165  // trySplitAndSortUpdateEvent try to split update events if unique key is updated
  1166  // returns true if some updated events is split
  1167  func trySplitAndSortUpdateEvent(
  1168  	events []*RowChangedEvent,
  1169  ) ([]*RowChangedEvent, error) {
  1170  	rowChangedEvents := make([]*RowChangedEvent, 0, len(events))
  1171  	split := false
  1172  	for _, e := range events {
  1173  		if e == nil {
  1174  			log.Warn("skip emit nil event",
  1175  				zap.Any("event", e))
  1176  			continue
  1177  		}
  1178  
  1179  		colLen := len(e.Columns)
  1180  		preColLen := len(e.PreColumns)
  1181  		// Some transactions could generate empty row change event, such as
  1182  		// begin; insert into t (id) values (1); delete from t where id=1; commit;
  1183  		// Just ignore these row changed events.
  1184  		if colLen == 0 && preColLen == 0 {
  1185  			log.Warn("skip emit empty row event",
  1186  				zap.Any("event", e))
  1187  			continue
  1188  		}
  1189  
  1190  		// This indicates that it is an update event. if the pk or uk is updated,
  1191  		// we need to split it into two events (delete and insert).
  1192  		if e.IsUpdate() && ShouldSplitUpdateEvent(e) {
  1193  			deleteEvent, insertEvent, err := SplitUpdateEvent(e)
  1194  			if err != nil {
  1195  				return nil, errors.Trace(err)
  1196  			}
  1197  			split = true
  1198  			rowChangedEvents = append(rowChangedEvents, deleteEvent, insertEvent)
  1199  		} else {
  1200  			rowChangedEvents = append(rowChangedEvents, e)
  1201  		}
  1202  	}
  1203  	// some updated events is split, need to sort
  1204  	if split {
  1205  		sort.Sort(txnRows(rowChangedEvents))
  1206  	}
  1207  	return rowChangedEvents, nil
  1208  }
  1209  
  1210  func isNonEmptyUniqueOrHandleCol(col *ColumnData, tableInfo *TableInfo) bool {
  1211  	if col != nil {
  1212  		colFlag := tableInfo.ForceGetColumnFlagType(col.ColumnID)
  1213  		return colFlag.IsUniqueKey() || colFlag.IsHandleKey()
  1214  	}
  1215  	return false
  1216  }
  1217  
  1218  // ShouldSplitUpdateEvent determines if the split event is needed to align the old format based on
  1219  // whether the handle key column or unique key has been modified.
  1220  // If  is modified, we need to use splitUpdateEvent to split the update event into a delete and an insert event.
  1221  func ShouldSplitUpdateEvent(updateEvent *RowChangedEvent) bool {
  1222  	// nil event will never be split.
  1223  	if updateEvent == nil {
  1224  		return false
  1225  	}
  1226  
  1227  	tableInfo := updateEvent.TableInfo
  1228  	for i := range updateEvent.Columns {
  1229  		col := updateEvent.Columns[i]
  1230  		preCol := updateEvent.PreColumns[i]
  1231  		if isNonEmptyUniqueOrHandleCol(col, tableInfo) && isNonEmptyUniqueOrHandleCol(preCol, tableInfo) {
  1232  			colValueString := ColumnValueString(col.Value)
  1233  			preColValueString := ColumnValueString(preCol.Value)
  1234  			// If one unique key columns is updated, we need to split the event row.
  1235  			if colValueString != preColValueString {
  1236  				return true
  1237  			}
  1238  		}
  1239  	}
  1240  	return false
  1241  }
  1242  
  1243  // SplitUpdateEvent splits an update event into a delete and an insert event.
  1244  func SplitUpdateEvent(
  1245  	updateEvent *RowChangedEvent,
  1246  ) (*RowChangedEvent, *RowChangedEvent, error) {
  1247  	if updateEvent == nil {
  1248  		return nil, nil, errors.New("nil event cannot be split")
  1249  	}
  1250  
  1251  	// If there is an update to handle key columns,
  1252  	// we need to split the event into two events to be compatible with the old format.
  1253  	// NOTICE: Here we don't need a full deep copy because
  1254  	// our two events need Columns and PreColumns respectively,
  1255  	// so it won't have an impact and no more full deep copy wastes memory.
  1256  	deleteEvent := *updateEvent
  1257  	deleteEvent.Columns = nil
  1258  
  1259  	insertEvent := *updateEvent
  1260  	// NOTICE: clean up pre cols for insert event.
  1261  	insertEvent.PreColumns = nil
  1262  
  1263  	return &deleteEvent, &insertEvent, nil
  1264  }
  1265  
  1266  // Append adds a row changed event into SingleTableTxn
  1267  func (t *SingleTableTxn) Append(row *RowChangedEvent) {
  1268  	if row.StartTs != t.StartTs || row.CommitTs != t.CommitTs || row.PhysicalTableID != t.GetPhysicalTableID() {
  1269  		log.Panic("unexpected row change event",
  1270  			zap.Uint64("startTs", t.StartTs),
  1271  			zap.Uint64("commitTs", t.CommitTs),
  1272  			zap.Any("table", t.GetPhysicalTableID()),
  1273  			zap.Any("row", row))
  1274  	}
  1275  	t.Rows = append(t.Rows, row)
  1276  }
  1277  
  1278  // TopicPartitionKey contains the topic and partition key of the message.
  1279  type TopicPartitionKey struct {
  1280  	Topic          string
  1281  	Partition      int32
  1282  	PartitionKey   string
  1283  	TotalPartition int32
  1284  }