github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/entry/mounter.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 entry
    15  
    16  import (
    17  	"bytes"
    18  	"context"
    19  	"encoding/hex"
    20  	"encoding/json"
    21  	"fmt"
    22  	"math"
    23  	"sort"
    24  	"time"
    25  	"unsafe"
    26  
    27  	"github.com/pingcap/errors"
    28  	"github.com/pingcap/log"
    29  	"github.com/pingcap/tidb/pkg/kv"
    30  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    31  	"github.com/pingcap/tidb/pkg/parser/mysql"
    32  	"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
    33  	"github.com/pingcap/tidb/pkg/table"
    34  	"github.com/pingcap/tidb/pkg/tablecodec"
    35  	"github.com/pingcap/tidb/pkg/types"
    36  	"github.com/pingcap/tidb/pkg/util/rowcodec"
    37  	"github.com/pingcap/tiflow/cdc/model"
    38  	cerror "github.com/pingcap/tiflow/pkg/errors"
    39  	pfilter "github.com/pingcap/tiflow/pkg/filter"
    40  	"github.com/pingcap/tiflow/pkg/integrity"
    41  	"github.com/pingcap/tiflow/pkg/spanz"
    42  	"github.com/prometheus/client_golang/prometheus"
    43  	"go.uber.org/zap"
    44  )
    45  
    46  type baseKVEntry struct {
    47  	StartTs uint64
    48  	// Commit or resolved TS
    49  	CRTs uint64
    50  
    51  	PhysicalTableID int64
    52  	RecordID        kv.Handle
    53  	Delete          bool
    54  }
    55  
    56  type rowKVEntry struct {
    57  	baseKVEntry
    58  	Row    map[int64]types.Datum
    59  	PreRow map[int64]types.Datum
    60  
    61  	// In some cases, row data may exist but not contain any Datum,
    62  	// use this RowExist/PreRowExist variable to distinguish between row data that does not exist
    63  	// or row data that does not contain any Datum.
    64  	RowExist    bool
    65  	PreRowExist bool
    66  }
    67  
    68  // DDLTableInfo contains the tableInfo about tidb_ddl_job and tidb_ddl_history
    69  // and the column id of `job_meta` in these two tables.
    70  type DDLTableInfo struct {
    71  	// ddlJobsTable use to parse all ddl jobs except `create table`
    72  	DDLJobTable *model.TableInfo
    73  	// It holds the column id of `job_meta` in table `tidb_ddl_jobs`.
    74  	JobMetaColumnIDinJobTable int64
    75  	// ddlHistoryTable only use to parse `create table` ddl job
    76  	DDLHistoryTable *model.TableInfo
    77  	// It holds the column id of `job_meta` in table `tidb_ddl_history`.
    78  	JobMetaColumnIDinHistoryTable int64
    79  }
    80  
    81  // Mounter is used to parse SQL events from KV events
    82  type Mounter interface {
    83  	// DecodeEvent accepts `model.PolymorphicEvent` with `RawKVEntry` filled and
    84  	// decodes `RawKVEntry` into `RowChangedEvent`.
    85  	// If a `model.PolymorphicEvent` should be ignored, it will returns (false, nil).
    86  	DecodeEvent(ctx context.Context, event *model.PolymorphicEvent) error
    87  }
    88  
    89  type mounter struct {
    90  	schemaStorage                SchemaStorage
    91  	tz                           *time.Location
    92  	changefeedID                 model.ChangeFeedID
    93  	filter                       pfilter.Filter
    94  	metricTotalRows              prometheus.Gauge
    95  	metricIgnoredDMLEventCounter prometheus.Counter
    96  
    97  	integrity *integrity.Config
    98  
    99  	// decoder and preDecoder are used to decode the raw value, also used to extract checksum,
   100  	// they should not be nil after decode at least one event in the row format v2.
   101  	decoder    *rowcodec.DatumMapDecoder
   102  	preDecoder *rowcodec.DatumMapDecoder
   103  
   104  	// encoder is used to calculate the checksum.
   105  	encoder *rowcodec.Encoder
   106  	// sctx hold some information can be used by the encoder to calculate the checksum.
   107  	sctx *stmtctx.StatementContext
   108  }
   109  
   110  // NewMounter creates a mounter
   111  func NewMounter(schemaStorage SchemaStorage,
   112  	changefeedID model.ChangeFeedID,
   113  	tz *time.Location,
   114  	filter pfilter.Filter,
   115  	integrity *integrity.Config,
   116  ) Mounter {
   117  	return &mounter{
   118  		schemaStorage: schemaStorage,
   119  		changefeedID:  changefeedID,
   120  		filter:        filter,
   121  		metricTotalRows: totalRowsCountGauge.
   122  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   123  		metricIgnoredDMLEventCounter: ignoredDMLEventCounter.
   124  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   125  		tz:        tz,
   126  		integrity: integrity,
   127  
   128  		encoder: &rowcodec.Encoder{},
   129  		sctx:    stmtctx.NewStmtCtxWithTimeZone(tz),
   130  	}
   131  }
   132  
   133  // DecodeEvent decode kv events using ddl puller's schemaStorage
   134  // this method could block indefinitely if the DDL puller is lagging.
   135  func (m *mounter) DecodeEvent(ctx context.Context, event *model.PolymorphicEvent) error {
   136  	m.metricTotalRows.Inc()
   137  	if event.IsResolved() {
   138  		return nil
   139  	}
   140  	row, err := m.unmarshalAndMountRowChanged(ctx, event.RawKV)
   141  	if err != nil {
   142  		return errors.Trace(err)
   143  	}
   144  
   145  	event.Row = row
   146  	event.RawKV.Value = nil
   147  	event.RawKV.OldValue = nil
   148  
   149  	m.decoder = nil
   150  	m.preDecoder = nil
   151  
   152  	return nil
   153  }
   154  
   155  func (m *mounter) unmarshalAndMountRowChanged(ctx context.Context, raw *model.RawKVEntry) (*model.RowChangedEvent, error) {
   156  	if !bytes.HasPrefix(raw.Key, tablePrefix) {
   157  		return nil, nil
   158  	}
   159  	key, physicalTableID, err := decodeTableID(raw.Key)
   160  	if err != nil {
   161  		return nil, err
   162  	}
   163  	if len(raw.OldValue) == 0 && len(raw.Value) == 0 {
   164  		log.Warn("empty value and old value",
   165  			zap.String("namespace", m.changefeedID.Namespace),
   166  			zap.String("changefeed", m.changefeedID.ID),
   167  			zap.Any("row", raw))
   168  	}
   169  	baseInfo := baseKVEntry{
   170  		StartTs:         raw.StartTs,
   171  		CRTs:            raw.CRTs,
   172  		PhysicalTableID: physicalTableID,
   173  		Delete:          raw.OpType == model.OpTypeDelete,
   174  	}
   175  	// When async commit is enabled, the commitTs of DMLs may be equals with DDL finishedTs.
   176  	// A DML whose commitTs is equal to a DDL finishedTs should use the schema info before the DDL.
   177  	snap, err := m.schemaStorage.GetSnapshot(ctx, raw.CRTs-1)
   178  	if err != nil {
   179  		return nil, errors.Trace(err)
   180  	}
   181  	row, err := func() (*model.RowChangedEvent, error) {
   182  		if snap.IsIneligibleTableID(physicalTableID) {
   183  			log.Debug("skip the DML of ineligible table", zap.Uint64("ts", raw.CRTs), zap.Int64("tableID", physicalTableID))
   184  			return nil, nil
   185  		}
   186  		tableInfo, exist := snap.PhysicalTableByID(physicalTableID)
   187  		if !exist {
   188  			// for truncate table and truncate table partition DDL, the table ID is changed, but DML can be inserted to TiKV with old table ID.
   189  			// normally, cdc will close the old table pipeline and create a new one, and these invalid DMLs keys will not be pulled by CDC,
   190  			// but if redo is enabled or push based table pipeline is enabled, puller and mounter are not blocked by barrier ts.
   191  			// So some invalid DML keys will be decoded before processor removing the table pipeline
   192  			if snap.IsTruncateTableID(physicalTableID) {
   193  				log.Debug("skip the DML of truncated table", zap.Uint64("ts", raw.CRTs), zap.Int64("tableID", physicalTableID))
   194  				return nil, nil
   195  			}
   196  			log.Error("can not found table schema",
   197  				zap.Uint64("ts", raw.CRTs),
   198  				zap.String("key", hex.EncodeToString(raw.Key)),
   199  				zap.Int64("tableID", physicalTableID))
   200  			return nil, cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(physicalTableID)
   201  		}
   202  		if bytes.HasPrefix(key, recordPrefix) {
   203  			rowKV, err := m.unmarshalRowKVEntry(tableInfo, raw.Key, raw.Value, raw.OldValue, baseInfo)
   204  			if err != nil {
   205  				return nil, errors.Trace(err)
   206  			}
   207  			if rowKV == nil {
   208  				return nil, nil
   209  			}
   210  			row, rawRow, err := m.mountRowKVEntry(tableInfo, rowKV, raw.ApproximateDataSize())
   211  			if err != nil {
   212  				return nil, err
   213  			}
   214  			// We need to filter a row here because we need its tableInfo.
   215  			ignore, err := m.filter.ShouldIgnoreDMLEvent(row, rawRow, tableInfo)
   216  			if err != nil {
   217  				return nil, err
   218  			}
   219  			// TODO(dongmen): try to find better way to indicate this row has been filtered.
   220  			// Return a nil RowChangedEvent if this row should be ignored.
   221  			if ignore {
   222  				m.metricIgnoredDMLEventCounter.Inc()
   223  				return nil, nil
   224  			}
   225  			return row, nil
   226  		}
   227  		return nil, nil
   228  	}()
   229  	if err != nil && !cerror.ShouldFailChangefeed(err) {
   230  		log.Error("failed to mount and unmarshals entry, start to print debug info", zap.Error(err))
   231  		snap.PrintStatus(log.Error)
   232  	}
   233  	return row, err
   234  }
   235  
   236  func (m *mounter) unmarshalRowKVEntry(
   237  	tableInfo *model.TableInfo,
   238  	rawKey []byte,
   239  	rawValue []byte,
   240  	rawOldValue []byte,
   241  	base baseKVEntry,
   242  ) (*rowKVEntry, error) {
   243  	recordID, err := tablecodec.DecodeRowKey(rawKey)
   244  	if err != nil {
   245  		return nil, errors.Trace(err)
   246  	}
   247  	base.RecordID = recordID
   248  
   249  	var (
   250  		row, preRow           map[int64]types.Datum
   251  		rowExist, preRowExist bool
   252  	)
   253  
   254  	row, rowExist, err = m.decodeRow(rawValue, recordID, tableInfo, false)
   255  	if err != nil {
   256  		return nil, errors.Trace(err)
   257  	}
   258  
   259  	preRow, preRowExist, err = m.decodeRow(rawOldValue, recordID, tableInfo, true)
   260  	if err != nil {
   261  		return nil, errors.Trace(err)
   262  	}
   263  
   264  	return &rowKVEntry{
   265  		baseKVEntry: base,
   266  		Row:         row,
   267  		PreRow:      preRow,
   268  		RowExist:    rowExist,
   269  		PreRowExist: preRowExist,
   270  	}, nil
   271  }
   272  
   273  func (m *mounter) decodeRow(
   274  	rawValue []byte, recordID kv.Handle, tableInfo *model.TableInfo, isPreColumns bool,
   275  ) (map[int64]types.Datum, bool, error) {
   276  	if len(rawValue) == 0 {
   277  		return map[int64]types.Datum{}, false, nil
   278  	}
   279  	handleColIDs, handleColFt, reqCols := tableInfo.GetRowColInfos()
   280  	var (
   281  		datums map[int64]types.Datum
   282  		err    error
   283  	)
   284  
   285  	if rowcodec.IsNewFormat(rawValue) {
   286  		decoder := rowcodec.NewDatumMapDecoder(reqCols, m.tz)
   287  		if isPreColumns {
   288  			m.preDecoder = decoder
   289  		} else {
   290  			m.decoder = decoder
   291  		}
   292  		datums, err = decodeRowV2(decoder, rawValue)
   293  	} else {
   294  		datums, err = decodeRowV1(rawValue, tableInfo, m.tz)
   295  	}
   296  
   297  	if err != nil {
   298  		return nil, false, errors.Trace(err)
   299  	}
   300  
   301  	datums, err = tablecodec.DecodeHandleToDatumMap(
   302  		recordID, handleColIDs, handleColFt, m.tz, datums)
   303  	if err != nil {
   304  		return nil, false, errors.Trace(err)
   305  	}
   306  
   307  	return datums, true, nil
   308  }
   309  
   310  // IsLegacyFormatJob returns true if the job is from the legacy DDL list key.
   311  func IsLegacyFormatJob(rawKV *model.RawKVEntry) bool {
   312  	return bytes.HasPrefix(rawKV.Key, metaPrefix)
   313  }
   314  
   315  // ParseDDLJob parses the job from the raw KV entry.
   316  func ParseDDLJob(rawKV *model.RawKVEntry, ddlTableInfo *DDLTableInfo) (*timodel.Job, error) {
   317  	var v []byte
   318  	var datum types.Datum
   319  
   320  	// for test case only
   321  	if bytes.HasPrefix(rawKV.Key, metaPrefix) {
   322  		v = rawKV.Value
   323  		job, err := parseJob(v, rawKV.StartTs, rawKV.CRTs, false)
   324  		if err != nil || job == nil {
   325  			job, err = parseJob(v, rawKV.StartTs, rawKV.CRTs, true)
   326  		}
   327  		return job, err
   328  	}
   329  
   330  	recordID, err := tablecodec.DecodeRowKey(rawKV.Key)
   331  	if err != nil {
   332  		return nil, errors.Trace(err)
   333  	}
   334  
   335  	tableID := tablecodec.DecodeTableID(rawKV.Key)
   336  
   337  	// parse it with tidb_ddl_job
   338  	if tableID == spanz.JobTableID {
   339  		row, err := decodeRow(rawKV.Value, recordID, ddlTableInfo.DDLJobTable, time.UTC)
   340  		if err != nil {
   341  			return nil, errors.Trace(err)
   342  		}
   343  		datum = row[ddlTableInfo.JobMetaColumnIDinJobTable]
   344  		v = datum.GetBytes()
   345  
   346  		return parseJob(v, rawKV.StartTs, rawKV.CRTs, false)
   347  	} else if tableID == spanz.JobHistoryID {
   348  		// parse it with tidb_ddl_history
   349  		row, err := decodeRow(rawKV.Value, recordID, ddlTableInfo.DDLHistoryTable, time.UTC)
   350  		if err != nil {
   351  			return nil, errors.Trace(err)
   352  		}
   353  		datum = row[ddlTableInfo.JobMetaColumnIDinHistoryTable]
   354  		v = datum.GetBytes()
   355  
   356  		return parseJob(v, rawKV.StartTs, rawKV.CRTs, true)
   357  	}
   358  
   359  	return nil, fmt.Errorf("Unvalid tableID %v in rawKV.Key", tableID)
   360  }
   361  
   362  // parseJob unmarshal the job from "v".
   363  // fromHistoryTable is used to distinguish the job is from tidb_dd_job or tidb_ddl_history
   364  // We need to be compatible with the two modes, enable_fast_create_table=on and enable_fast_create_table=off
   365  // When enable_fast_create_table=on, `create table` will only be inserted into tidb_ddl_history after being executed successfully.
   366  // When enable_fast_create_table=off, `create table` just like other ddls will be firstly inserted to tidb_ddl_job,
   367  // and being inserted into tidb_ddl_history after being executed successfully.
   368  // In both two modes, other ddls are all firstly inserted into tidb_ddl_job, and then inserted into tidb_ddl_history after being executed successfully.
   369  //
   370  // To be compatible with these two modes, we will get `create table` ddl from tidb_ddl_history, and all ddls from tidb_ddl_job.
   371  // When enable_fast_create_table=off, for each `create table` ddl we will get twice(once from tidb_ddl_history, once from tidb_ddl_job)
   372  // Because in `handleJob` we will skip the repeated ddls, thus it's ok for us to get `create table` twice.
   373  // Besides, the `create table` from tidb_ddl_job always have a earlier commitTs than from tidb_ddl_history.
   374  // Therefore, we always use the commitTs of ddl from `tidb_ddl_job` as StartTs, which ensures we can get all the dmls.
   375  func parseJob(v []byte, startTs, CRTs uint64, fromHistoryTable bool) (*timodel.Job, error) {
   376  	var job timodel.Job
   377  	err := json.Unmarshal(v, &job)
   378  	if err != nil {
   379  		return nil, errors.Trace(err)
   380  	}
   381  
   382  	if fromHistoryTable {
   383  		// we only want to get `create table` ddl from tidb_ddl_history, so we just throw out others ddls.
   384  		// We only want the job with `JobStateSynced`, which is means the ddl job is done successfully.
   385  		// Besides, to satisfy the subsequent processing,
   386  		// We need to set the job to be Done to make it will replay in schemaStorage
   387  		if job.Type != timodel.ActionCreateTable || job.State != timodel.JobStateSynced {
   388  			return nil, nil
   389  		}
   390  		job.State = timodel.JobStateDone
   391  	} else {
   392  		// we need to get all ddl job which is done from tidb_ddl_job
   393  		if !job.IsDone() {
   394  			return nil, nil
   395  		}
   396  	}
   397  
   398  	// FinishedTS is only set when the job is synced,
   399  	// but we can use the entry's ts here
   400  	job.StartTS = startTs
   401  	// Since ddl in stateDone doesn't contain the FinishedTS,
   402  	// we need to set it as the txn's commit ts.
   403  	job.BinlogInfo.FinishedTS = CRTs
   404  	return &job, nil
   405  }
   406  
   407  func datum2Column(
   408  	tableInfo *model.TableInfo, datums map[int64]types.Datum, tz *time.Location,
   409  ) ([]*model.ColumnData, []types.Datum, []*timodel.ColumnInfo, error) {
   410  	cols := make([]*model.ColumnData, len(tableInfo.RowColumnsOffset))
   411  	rawCols := make([]types.Datum, len(tableInfo.RowColumnsOffset))
   412  
   413  	// columnInfos should have the same length and order with cols
   414  	columnInfos := make([]*timodel.ColumnInfo, len(tableInfo.RowColumnsOffset))
   415  
   416  	for _, colInfo := range tableInfo.Columns {
   417  		if !model.IsColCDCVisible(colInfo) {
   418  			log.Debug("skip the column which is not visible",
   419  				zap.String("table", tableInfo.Name.O), zap.String("column", colInfo.Name.O))
   420  			continue
   421  		}
   422  
   423  		colID := colInfo.ID
   424  		colDatum, exist := datums[colID]
   425  
   426  		var (
   427  			colValue interface{}
   428  			size     int
   429  			warn     string
   430  			err      error
   431  		)
   432  		if exist {
   433  			colValue, size, warn, err = formatColVal(colDatum, colInfo)
   434  		} else {
   435  			colDatum, colValue, size, warn, err = getDefaultOrZeroValue(colInfo, tz)
   436  		}
   437  		if err != nil {
   438  			return nil, nil, nil, errors.Trace(err)
   439  		}
   440  		if warn != "" {
   441  			log.Warn(warn, zap.String("table", tableInfo.TableName.String()),
   442  				zap.String("column", colInfo.Name.String()))
   443  		}
   444  
   445  		offset := tableInfo.RowColumnsOffset[colID]
   446  		rawCols[offset] = colDatum
   447  		cols[offset] = &model.ColumnData{
   448  			ColumnID: colID,
   449  			Value:    colValue,
   450  			// ApproximateBytes = column data size + column struct size
   451  			ApproximateBytes: size + sizeOfEmptyColumn,
   452  		}
   453  		columnInfos[offset] = colInfo
   454  	}
   455  	return cols, rawCols, columnInfos, nil
   456  }
   457  
   458  func (m *mounter) calculateChecksum(
   459  	columnInfos []*timodel.ColumnInfo, rawColumns []types.Datum,
   460  ) (uint32, error) {
   461  	columns := make([]rowcodec.ColData, 0, len(rawColumns))
   462  	for idx, col := range columnInfos {
   463  		column := rowcodec.ColData{
   464  			ColumnInfo: col,
   465  			Datum:      &rawColumns[idx],
   466  		}
   467  		columns = append(columns, column)
   468  	}
   469  	sort.Slice(columns, func(i, j int) bool {
   470  		return columns[i].ID < columns[j].ID
   471  	})
   472  
   473  	calculator := rowcodec.RowData{
   474  		Cols: columns,
   475  		Data: make([]byte, 0),
   476  	}
   477  
   478  	checksum, err := calculator.Checksum(m.tz)
   479  	if err != nil {
   480  		return 0, errors.Trace(err)
   481  	}
   482  	return checksum, nil
   483  }
   484  
   485  // return error when calculate the checksum failed.
   486  // return false if the checksum is not matched
   487  func (m *mounter) verifyChecksum(
   488  	columnInfos []*timodel.ColumnInfo, rawColumns []types.Datum, isPreRow bool,
   489  ) (uint32, bool, error) {
   490  	if !m.integrity.Enabled() {
   491  		return 0, true, nil
   492  	}
   493  
   494  	var decoder *rowcodec.DatumMapDecoder
   495  	if isPreRow {
   496  		decoder = m.preDecoder
   497  	} else {
   498  		decoder = m.decoder
   499  	}
   500  
   501  	// if the checksum cannot be found, which means the upstream TiDB checksum is not enabled,
   502  	// so return matched as true to skip check the event.
   503  	first, ok := decoder.GetChecksum()
   504  	if !ok {
   505  		return 0, true, nil
   506  	}
   507  
   508  	checksum, err := m.calculateChecksum(columnInfos, rawColumns)
   509  	if err != nil {
   510  		log.Error("failed to calculate the checksum", zap.Uint32("first", first), zap.Error(err))
   511  		return 0, false, err
   512  	}
   513  
   514  	// the first checksum matched, it hits in the most case.
   515  	if checksum == first {
   516  		log.Debug("checksum matched",
   517  			zap.Uint32("checksum", checksum), zap.Uint32("first", first))
   518  		return checksum, true, nil
   519  	}
   520  
   521  	extra, ok := decoder.GetExtraChecksum()
   522  	if !ok {
   523  		log.Error("cannot found the extra checksum, the first checksum mismatched",
   524  			zap.Uint32("checksum", checksum),
   525  			zap.Uint32("first", first))
   526  		return checksum, false, nil
   527  	}
   528  
   529  	if checksum == extra {
   530  		log.Debug("extra checksum matched, this may happen the upstream TiDB is during the DDL execution phase",
   531  			zap.Uint32("checksum", checksum),
   532  			zap.Uint32("extra", extra))
   533  		return checksum, true, nil
   534  	}
   535  
   536  	log.Error("checksum mismatch",
   537  		zap.Uint32("checksum", checksum),
   538  		zap.Uint32("first", first),
   539  		zap.Uint32("extra", extra))
   540  	return checksum, false, nil
   541  }
   542  
   543  func (m *mounter) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntry, dataSize int64) (*model.RowChangedEvent, model.RowChangedDatums, error) {
   544  	var (
   545  		rawRow      model.RowChangedDatums
   546  		columnInfos []*timodel.ColumnInfo
   547  		matched     bool
   548  		err         error
   549  
   550  		checksum *integrity.Checksum
   551  
   552  		checksumVersion int
   553  		corrupted       bool
   554  	)
   555  
   556  	if m.decoder != nil {
   557  		checksumVersion = m.decoder.ChecksumVersion()
   558  	} else if m.preDecoder != nil {
   559  		checksumVersion = m.preDecoder.ChecksumVersion()
   560  	}
   561  
   562  	// Decode previous columns.
   563  	var (
   564  		preCols     []*model.ColumnData
   565  		preRawCols  []types.Datum
   566  		preChecksum uint32
   567  	)
   568  	if row.PreRowExist {
   569  		// FIXME(leoppro): using pre table info to mounter pre column datum
   570  		// the pre column and current column in one event may using different table info
   571  		preCols, preRawCols, columnInfos, err = datum2Column(tableInfo, row.PreRow, m.tz)
   572  		if err != nil {
   573  			return nil, rawRow, errors.Trace(err)
   574  		}
   575  
   576  		preChecksum, matched, err = m.verifyChecksum(columnInfos, preRawCols, true)
   577  		if err != nil {
   578  			log.Error("calculate the previous columns checksum failed",
   579  				zap.Any("tableInfo", tableInfo),
   580  				zap.Any("rawCols", preRawCols))
   581  			return nil, rawRow, errors.Trace(err)
   582  		}
   583  
   584  		if !matched {
   585  			log.Error("previous columns checksum mismatch",
   586  				zap.Uint32("checksum", preChecksum),
   587  				zap.Any("tableInfo", tableInfo),
   588  				zap.Any("rawCols", preRawCols))
   589  			if m.integrity.ErrorHandle() {
   590  				return nil, rawRow, cerror.ErrCorruptedDataMutation.
   591  					GenWithStackByArgs(m.changefeedID.Namespace, m.changefeedID.ID)
   592  			}
   593  			corrupted = true
   594  		}
   595  	}
   596  
   597  	var (
   598  		cols            []*model.ColumnData
   599  		rawCols         []types.Datum
   600  		currentChecksum uint32
   601  	)
   602  	if row.RowExist {
   603  		cols, rawCols, columnInfos, err = datum2Column(tableInfo, row.Row, m.tz)
   604  		if err != nil {
   605  			return nil, rawRow, errors.Trace(err)
   606  		}
   607  
   608  		currentChecksum, matched, err = m.verifyChecksum(columnInfos, rawCols, false)
   609  		if err != nil {
   610  			log.Error("calculate the current columns checksum failed",
   611  				zap.Any("tableInfo", tableInfo),
   612  				zap.Any("rawCols", rawCols))
   613  			return nil, rawRow, errors.Trace(err)
   614  		}
   615  		if !matched {
   616  			log.Error("current columns checksum mismatch",
   617  				zap.Uint32("checksum", currentChecksum),
   618  				zap.Any("tableInfo", tableInfo),
   619  				zap.Any("rawCols", rawCols))
   620  			if m.integrity.ErrorHandle() {
   621  				return nil, rawRow, cerror.ErrCorruptedDataMutation.
   622  					GenWithStackByArgs(m.changefeedID.Namespace, m.changefeedID.ID)
   623  			}
   624  			corrupted = true
   625  		}
   626  	}
   627  
   628  	var intRowID int64
   629  	if row.RecordID.IsInt() {
   630  		intRowID = row.RecordID.IntValue()
   631  	}
   632  
   633  	rawRow.PreRowDatums = preRawCols
   634  	rawRow.RowDatums = rawCols
   635  
   636  	// if both are 0, it means the checksum is not enabled
   637  	// so the checksum is nil to reduce memory allocation.
   638  	if preChecksum != 0 || currentChecksum != 0 {
   639  		checksum = &integrity.Checksum{
   640  			Current:   currentChecksum,
   641  			Previous:  preChecksum,
   642  			Corrupted: corrupted,
   643  			Version:   checksumVersion,
   644  		}
   645  	}
   646  
   647  	return &model.RowChangedEvent{
   648  		StartTs:         row.StartTs,
   649  		CommitTs:        row.CRTs,
   650  		RowID:           intRowID,
   651  		HandleKey:       row.RecordID,
   652  		PhysicalTableID: row.PhysicalTableID,
   653  		TableInfo:       tableInfo,
   654  		Columns:         cols,
   655  		PreColumns:      preCols,
   656  
   657  		Checksum: checksum,
   658  
   659  		ApproximateDataSize: dataSize,
   660  	}, rawRow, nil
   661  }
   662  
   663  var emptyBytes = make([]byte, 0)
   664  
   665  const (
   666  	sizeOfEmptyColumn = int(unsafe.Sizeof(model.Column{}))
   667  	sizeOfEmptyBytes  = int(unsafe.Sizeof(emptyBytes))
   668  	sizeOfEmptyString = int(unsafe.Sizeof(""))
   669  )
   670  
   671  func sizeOfDatum(d types.Datum) int {
   672  	array := [...]types.Datum{d}
   673  	return int(types.EstimatedMemUsage(array[:], 1))
   674  }
   675  
   676  func sizeOfString(s string) int {
   677  	// string data size + string struct size.
   678  	return len(s) + sizeOfEmptyString
   679  }
   680  
   681  func sizeOfBytes(b []byte) int {
   682  	// bytes data size + bytes struct size.
   683  	return len(b) + sizeOfEmptyBytes
   684  }
   685  
   686  // formatColVal return interface{} need to meet the same requirement as getDefaultOrZeroValue
   687  func formatColVal(datum types.Datum, col *timodel.ColumnInfo) (
   688  	value interface{}, size int, warn string, err error,
   689  ) {
   690  	if datum.IsNull() {
   691  		return nil, 0, "", nil
   692  	}
   693  	switch col.GetType() {
   694  	case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp:
   695  		v := datum.GetMysqlTime().String()
   696  		return v, sizeOfString(v), "", nil
   697  	case mysql.TypeDuration:
   698  		v := datum.GetMysqlDuration().String()
   699  		return v, sizeOfString(v), "", nil
   700  	case mysql.TypeJSON:
   701  		v := datum.GetMysqlJSON().String()
   702  		return v, sizeOfString(v), "", nil
   703  	case mysql.TypeNewDecimal:
   704  		d := datum.GetMysqlDecimal()
   705  		if d == nil {
   706  			// nil takes 0 byte.
   707  			return nil, 0, "", nil
   708  		}
   709  		v := d.String()
   710  		return v, sizeOfString(v), "", nil
   711  	case mysql.TypeEnum:
   712  		v := datum.GetMysqlEnum().Value
   713  		const sizeOfV = unsafe.Sizeof(v)
   714  		return v, int(sizeOfV), "", nil
   715  	case mysql.TypeSet:
   716  		v := datum.GetMysqlSet().Value
   717  		const sizeOfV = unsafe.Sizeof(v)
   718  		return v, int(sizeOfV), "", nil
   719  	case mysql.TypeBit:
   720  		// Encode bits as integers to avoid pingcap/tidb#10988 (which also affects MySQL itself)
   721  		v, err := datum.GetBinaryLiteral().ToInt(types.DefaultStmtNoWarningContext)
   722  		const sizeOfV = unsafe.Sizeof(v)
   723  		return v, int(sizeOfV), "", err
   724  	case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar,
   725  		mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob:
   726  		b := datum.GetBytes()
   727  		if b == nil {
   728  			b = emptyBytes
   729  		}
   730  		return b, sizeOfBytes(b), "", nil
   731  	case mysql.TypeFloat:
   732  		v := datum.GetFloat32()
   733  		if math.IsNaN(float64(v)) || math.IsInf(float64(v), 1) || math.IsInf(float64(v), -1) {
   734  			warn = fmt.Sprintf("the value is invalid in column: %f", v)
   735  			v = 0
   736  		}
   737  		const sizeOfV = unsafe.Sizeof(v)
   738  		return v, int(sizeOfV), warn, nil
   739  	case mysql.TypeDouble:
   740  		v := datum.GetFloat64()
   741  		if math.IsNaN(v) || math.IsInf(v, 1) || math.IsInf(v, -1) {
   742  			warn = fmt.Sprintf("the value is invalid in column: %f", v)
   743  			v = 0
   744  		}
   745  		const sizeOfV = unsafe.Sizeof(v)
   746  		return v, int(sizeOfV), warn, nil
   747  	default:
   748  		// NOTICE: GetValue() may return some types that go sql not support, which will cause sink DML fail
   749  		// Make specified convert upper if you need
   750  		// Go sql support type ref to: https://github.com/golang/go/blob/go1.17.4/src/database/sql/driver/types.go#L236
   751  		return datum.GetValue(), sizeOfDatum(datum), "", nil
   752  	}
   753  }
   754  
   755  // Scenarios when call this function:
   756  // (1) column define default null at creating + insert without explicit column
   757  // (2) alter table add column default xxx + old existing data
   758  // (3) amend + insert without explicit column + alter table add column default xxx
   759  // (4) online DDL drop column + data insert at state delete-only
   760  //
   761  // getDefaultOrZeroValue return interface{} need to meet to require type in
   762  // https://github.com/golang/go/blob/go1.17.4/src/database/sql/driver/types.go#L236
   763  // Supported type is: nil, basic type(Int, Int8,..., Float32, Float64, String), Slice(uint8), other types not support
   764  // TODO: Check default expr support
   765  func getDefaultOrZeroValue(
   766  	col *timodel.ColumnInfo, tz *time.Location,
   767  ) (types.Datum, any, int, string, error) {
   768  	var (
   769  		d   types.Datum
   770  		err error
   771  	)
   772  	// NOTICE: SHOULD use OriginDefaultValue here, more info pls ref to
   773  	// https://github.com/pingcap/tiflow/issues/4048
   774  	// FIXME: Too many corner cases may hit here, like type truncate, timezone
   775  	// (1) If this column is uk(no pk), will cause data inconsistency in Scenarios(2)
   776  	// (2) If not fix here, will cause data inconsistency in Scenarios(3) directly
   777  	// Ref: https://github.com/pingcap/tidb/blob/d2c352980a43bb593db81fd1db996f47af596d91/table/column.go#L489
   778  	if col.GetOriginDefaultValue() != nil {
   779  		datum := types.NewDatum(col.GetOriginDefaultValue())
   780  		d, err = datum.ConvertTo(types.DefaultStmtNoWarningContext, &col.FieldType)
   781  		if err != nil {
   782  			return d, d.GetValue(), sizeOfDatum(d), "", errors.Trace(err)
   783  		}
   784  		switch col.GetType() {
   785  		case mysql.TypeTimestamp:
   786  			t := d.GetMysqlTime()
   787  			err = t.ConvertTimeZone(time.UTC, tz)
   788  			if err != nil {
   789  				return d, d.GetValue(), sizeOfDatum(d), "", errors.Trace(err)
   790  			}
   791  			d.SetMysqlTime(t)
   792  		}
   793  	} else if !mysql.HasNotNullFlag(col.GetFlag()) {
   794  		// NOTICE: NotNullCheck need do after OriginDefaultValue check, as when TiDB meet "amend + add column default xxx",
   795  		// ref: https://github.com/pingcap/ticdc/issues/3929
   796  		// must use null if TiDB not write the column value when default value is null
   797  		// and the value is null, see https://github.com/pingcap/tidb/issues/9304
   798  		d = types.NewDatum(nil)
   799  	} else {
   800  		switch col.GetType() {
   801  		case mysql.TypeEnum:
   802  			// For enum type, if no default value and not null is set,
   803  			// the default value is the first element of the enum list
   804  			name := col.FieldType.GetElem(0)
   805  			enumValue, err := types.ParseEnumName(col.FieldType.GetElems(), name, col.GetCollate())
   806  			if err != nil {
   807  				return d, nil, 0, "", errors.Trace(err)
   808  			}
   809  			d = types.NewMysqlEnumDatum(enumValue)
   810  		case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar:
   811  			return d, emptyBytes, sizeOfEmptyBytes, "", nil
   812  		default:
   813  			d = table.GetZeroValue(col)
   814  			if d.IsNull() {
   815  				log.Error("meet unsupported column type", zap.String("columnInfo", col.FieldType.String()))
   816  			}
   817  		}
   818  	}
   819  	v, size, warn, err := formatColVal(d, col)
   820  	return d, v, size, warn, err
   821  }
   822  
   823  // DecodeTableID decodes the raw key to a table ID
   824  func DecodeTableID(key []byte) (model.TableID, error) {
   825  	_, physicalTableID, err := decodeTableID(key)
   826  	if err != nil {
   827  		return 0, errors.Trace(err)
   828  	}
   829  	return physicalTableID, nil
   830  }