github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/applier/redo.go (about)

     1  // Copyright 2021 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 applier
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"net/url"
    20  	"os"
    21  	"time"
    22  
    23  	"github.com/pingcap/log"
    24  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    25  	"github.com/pingcap/tiflow/cdc/model"
    26  	"github.com/pingcap/tiflow/cdc/model/codec"
    27  	"github.com/pingcap/tiflow/cdc/processor/memquota"
    28  	"github.com/pingcap/tiflow/cdc/redo/reader"
    29  	"github.com/pingcap/tiflow/cdc/sink/ddlsink"
    30  	ddlfactory "github.com/pingcap/tiflow/cdc/sink/ddlsink/factory"
    31  	dmlfactory "github.com/pingcap/tiflow/cdc/sink/dmlsink/factory"
    32  	"github.com/pingcap/tiflow/cdc/sink/tablesink"
    33  	"github.com/pingcap/tiflow/pkg/config"
    34  	"github.com/pingcap/tiflow/pkg/errors"
    35  	"github.com/pingcap/tiflow/pkg/pdutil"
    36  	"github.com/pingcap/tiflow/pkg/redo"
    37  	"github.com/pingcap/tiflow/pkg/sink/mysql"
    38  	"github.com/pingcap/tiflow/pkg/spanz"
    39  	"github.com/prometheus/client_golang/prometheus"
    40  	"go.uber.org/zap"
    41  	"golang.org/x/sync/errgroup"
    42  )
    43  
    44  const (
    45  	applierChangefeed = "redo-applier"
    46  	warnDuration      = 3 * time.Minute
    47  	flushWaitDuration = 200 * time.Millisecond
    48  )
    49  
    50  var (
    51  	// In the boundary case, non-idempotent DDLs will not be executed.
    52  	// TODO(CharlesCheung96): fix this
    53  	unsupportedDDL = map[timodel.ActionType]struct{}{
    54  		timodel.ActionExchangeTablePartition: {},
    55  	}
    56  	errApplyFinished = errors.New("apply finished, can exit safely")
    57  )
    58  
    59  // RedoApplierConfig is the configuration used by a redo log applier
    60  type RedoApplierConfig struct {
    61  	SinkURI string
    62  	Storage string
    63  	Dir     string
    64  }
    65  
    66  // RedoApplier implements a redo log applier
    67  type RedoApplier struct {
    68  	cfg            *RedoApplierConfig
    69  	rd             reader.RedoLogReader
    70  	updateSplitter *updateEventSplitter
    71  
    72  	ddlSink         ddlsink.Sink
    73  	appliedDDLCount uint64
    74  
    75  	memQuota     *memquota.MemQuota
    76  	pendingQuota uint64
    77  
    78  	// sinkFactory is used to create table sinks.
    79  	sinkFactory *dmlfactory.SinkFactory
    80  	// tableSinks is a map from tableID to table sink.
    81  	// We create it when we need it, and close it after we finish applying the redo logs.
    82  	tableSinks         map[model.TableID]tablesink.TableSink
    83  	tableResolvedTsMap map[model.TableID]*memquota.MemConsumeRecord
    84  	appliedLogCount    uint64
    85  
    86  	errCh chan error
    87  
    88  	// changefeedID is used to identify the changefeed that this applier belongs to.
    89  	// not used for now.
    90  	changefeedID model.ChangeFeedID
    91  }
    92  
    93  // NewRedoApplier creates a new RedoApplier instance
    94  func NewRedoApplier(cfg *RedoApplierConfig) *RedoApplier {
    95  	return &RedoApplier{
    96  		cfg:   cfg,
    97  		errCh: make(chan error, 1024),
    98  	}
    99  }
   100  
   101  // toLogReaderConfig is an adapter to translate from applier config to redo reader config
   102  // returns storageType, *reader.toLogReaderConfig and error
   103  func (rac *RedoApplierConfig) toLogReaderConfig() (string, *reader.LogReaderConfig, error) {
   104  	uri, err := url.Parse(rac.Storage)
   105  	if err != nil {
   106  		return "", nil, errors.WrapError(errors.ErrConsistentStorage, err)
   107  	}
   108  	if redo.IsLocalStorage(uri.Scheme) {
   109  		uri.Scheme = "file"
   110  	}
   111  	cfg := &reader.LogReaderConfig{
   112  		URI:                *uri,
   113  		Dir:                rac.Dir,
   114  		UseExternalStorage: redo.IsExternalStorage(uri.Scheme),
   115  	}
   116  	return uri.Scheme, cfg, nil
   117  }
   118  
   119  func (ra *RedoApplier) catchError(ctx context.Context) error {
   120  	for {
   121  		select {
   122  		case <-ctx.Done():
   123  			return nil
   124  		case err := <-ra.errCh:
   125  			return err
   126  		}
   127  	}
   128  }
   129  
   130  func (ra *RedoApplier) initSink(ctx context.Context) (err error) {
   131  	replicaConfig := config.GetDefaultReplicaConfig()
   132  	ra.sinkFactory, err = dmlfactory.New(ctx, ra.changefeedID, ra.cfg.SinkURI, replicaConfig, ra.errCh, nil)
   133  	if err != nil {
   134  		return err
   135  	}
   136  	ra.ddlSink, err = ddlfactory.New(ctx, ra.changefeedID, ra.cfg.SinkURI, replicaConfig)
   137  	if err != nil {
   138  		return err
   139  	}
   140  
   141  	ra.tableSinks = make(map[model.TableID]tablesink.TableSink)
   142  	ra.tableResolvedTsMap = make(map[model.TableID]*memquota.MemConsumeRecord)
   143  	return nil
   144  }
   145  
   146  func (ra *RedoApplier) bgReleaseQuota(ctx context.Context) error {
   147  	ticker := time.NewTicker(time.Second)
   148  	defer ticker.Stop()
   149  	for {
   150  		select {
   151  		case <-ctx.Done():
   152  			return errors.Trace(ctx.Err())
   153  		case <-ticker.C:
   154  			for tableID, tableSink := range ra.tableSinks {
   155  				checkpointTs := tableSink.GetCheckpointTs()
   156  				ra.memQuota.Release(spanz.TableIDToComparableSpan(tableID), checkpointTs)
   157  			}
   158  		}
   159  	}
   160  }
   161  
   162  func (ra *RedoApplier) consumeLogs(ctx context.Context) error {
   163  	checkpointTs, resolvedTs, err := ra.rd.ReadMeta(ctx)
   164  	if err != nil {
   165  		return err
   166  	}
   167  	log.Info("apply redo log starts",
   168  		zap.Uint64("checkpointTs", checkpointTs),
   169  		zap.Uint64("resolvedTs", resolvedTs))
   170  	if err := ra.initSink(ctx); err != nil {
   171  		return err
   172  	}
   173  	defer ra.sinkFactory.Close()
   174  
   175  	shouldApplyDDL := func(row *model.RowChangedEvent, ddl *model.DDLEvent) bool {
   176  		if ddl == nil {
   177  			return false
   178  		} else if row == nil {
   179  			// no more rows to apply
   180  			return true
   181  		}
   182  		// If all rows before the DDL (which means row.CommitTs <= ddl.CommitTs)
   183  		// are applied, we should apply this DDL.
   184  		return row.CommitTs > ddl.CommitTs
   185  	}
   186  
   187  	row, err := ra.updateSplitter.readNextRow(ctx)
   188  	if err != nil {
   189  		return err
   190  	}
   191  	ddl, err := ra.rd.ReadNextDDL(ctx)
   192  	if err != nil {
   193  		return err
   194  	}
   195  	for {
   196  		if row == nil && ddl == nil {
   197  			break
   198  		}
   199  		if shouldApplyDDL(row, ddl) {
   200  			if err := ra.applyDDL(ctx, ddl, checkpointTs); err != nil {
   201  				return err
   202  			}
   203  			if ddl, err = ra.rd.ReadNextDDL(ctx); err != nil {
   204  				return err
   205  			}
   206  		} else {
   207  			if err := ra.applyRow(row, checkpointTs); err != nil {
   208  				return err
   209  			}
   210  			if row, err = ra.updateSplitter.readNextRow(ctx); err != nil {
   211  				return err
   212  			}
   213  		}
   214  	}
   215  	// wait all tables to flush data
   216  	for tableID := range ra.tableResolvedTsMap {
   217  		if err := ra.waitTableFlush(ctx, tableID, resolvedTs); err != nil {
   218  			return err
   219  		}
   220  		ra.tableSinks[tableID].Close()
   221  	}
   222  
   223  	log.Info("apply redo log finishes",
   224  		zap.Uint64("appliedLogCount", ra.appliedLogCount),
   225  		zap.Uint64("appliedDDLCount", ra.appliedDDLCount),
   226  		zap.Uint64("currentCheckpoint", resolvedTs))
   227  	return errApplyFinished
   228  }
   229  
   230  func (ra *RedoApplier) resetQuota(rowSize uint64) error {
   231  	if rowSize >= config.DefaultChangefeedMemoryQuota || rowSize < ra.pendingQuota {
   232  		log.Panic("row size exceeds memory quota",
   233  			zap.Uint64("rowSize", rowSize),
   234  			zap.Uint64("memoryQuota", config.DefaultChangefeedMemoryQuota))
   235  	}
   236  
   237  	// flush all tables before acquire new quota
   238  	for tableID, tableRecord := range ra.tableResolvedTsMap {
   239  		if !tableRecord.ResolvedTs.IsBatchMode() {
   240  			log.Panic("table resolved ts should always be in batch mode when apply redo log")
   241  		}
   242  
   243  		if err := ra.tableSinks[tableID].UpdateResolvedTs(tableRecord.ResolvedTs); err != nil {
   244  			return err
   245  		}
   246  		ra.memQuota.Record(spanz.TableIDToComparableSpan(tableID),
   247  			tableRecord.ResolvedTs, tableRecord.Size)
   248  
   249  		// reset new record
   250  		ra.tableResolvedTsMap[tableID] = &memquota.MemConsumeRecord{
   251  			ResolvedTs: tableRecord.ResolvedTs.AdvanceBatch(),
   252  			Size:       0,
   253  		}
   254  	}
   255  
   256  	oldQuota := ra.pendingQuota
   257  	ra.pendingQuota = rowSize * mysql.DefaultMaxTxnRow
   258  	if ra.pendingQuota > config.DefaultChangefeedMemoryQuota {
   259  		ra.pendingQuota = config.DefaultChangefeedMemoryQuota
   260  	} else if ra.pendingQuota < 64*1024 {
   261  		ra.pendingQuota = 64 * 1024
   262  	}
   263  	return ra.memQuota.BlockAcquire(ra.pendingQuota - oldQuota)
   264  }
   265  
   266  func (ra *RedoApplier) applyDDL(
   267  	ctx context.Context, ddl *model.DDLEvent, checkpointTs uint64,
   268  ) error {
   269  	shouldSkip := func() bool {
   270  		if ddl.CommitTs == checkpointTs {
   271  			if _, ok := unsupportedDDL[ddl.Type]; ok {
   272  				log.Error("ignore unsupported DDL", zap.Any("ddl", ddl))
   273  				return true
   274  			}
   275  		}
   276  		if ddl.TableInfo == nil {
   277  			// Note this could omly happen when using old version of cdc, and the commit ts
   278  			// of the DDL should be equal to checkpoint ts or resolved ts.
   279  			log.Warn("ignore DDL without table info", zap.Any("ddl", ddl))
   280  			return true
   281  		}
   282  		return false
   283  	}
   284  	if shouldSkip() {
   285  		return nil
   286  	}
   287  	log.Warn("apply DDL", zap.Any("ddl", ddl))
   288  	// Wait all tables to flush data before applying DDL.
   289  	// TODO: only block tables that are affected by this DDL.
   290  	for tableID := range ra.tableSinks {
   291  		if err := ra.waitTableFlush(ctx, tableID, ddl.CommitTs); err != nil {
   292  			return err
   293  		}
   294  	}
   295  	if err := ra.ddlSink.WriteDDLEvent(ctx, ddl); err != nil {
   296  		return err
   297  	}
   298  	ra.appliedDDLCount++
   299  	return nil
   300  }
   301  
   302  func (ra *RedoApplier) applyRow(
   303  	row *model.RowChangedEvent, checkpointTs model.Ts,
   304  ) error {
   305  	rowSize := uint64(row.ApproximateBytes())
   306  	if rowSize > ra.pendingQuota {
   307  		if err := ra.resetQuota(uint64(row.ApproximateBytes())); err != nil {
   308  			return err
   309  		}
   310  	}
   311  	ra.pendingQuota -= rowSize
   312  
   313  	tableID := row.PhysicalTableID
   314  	if _, ok := ra.tableSinks[tableID]; !ok {
   315  		tableSink := ra.sinkFactory.CreateTableSink(
   316  			model.DefaultChangeFeedID(applierChangefeed),
   317  			spanz.TableIDToComparableSpan(tableID),
   318  			checkpointTs,
   319  			pdutil.NewClock4Test(),
   320  			prometheus.NewCounter(prometheus.CounterOpts{}),
   321  			prometheus.NewHistogram(prometheus.HistogramOpts{}),
   322  		)
   323  		ra.tableSinks[tableID] = tableSink
   324  	}
   325  	if _, ok := ra.tableResolvedTsMap[tableID]; !ok {
   326  		// Initialize table record using checkpointTs.
   327  		ra.tableResolvedTsMap[tableID] = &memquota.MemConsumeRecord{
   328  			ResolvedTs: model.ResolvedTs{
   329  				Mode:    model.BatchResolvedMode,
   330  				Ts:      checkpointTs,
   331  				BatchID: 1,
   332  			},
   333  			Size: 0,
   334  		}
   335  	}
   336  
   337  	ra.tableSinks[tableID].AppendRowChangedEvents(row)
   338  	record := ra.tableResolvedTsMap[tableID]
   339  	record.Size += rowSize
   340  	if row.CommitTs > record.ResolvedTs.Ts {
   341  		// Use batch resolvedTs to flush data as quickly as possible.
   342  		ra.tableResolvedTsMap[tableID] = &memquota.MemConsumeRecord{
   343  			ResolvedTs: model.ResolvedTs{
   344  				Mode:    model.BatchResolvedMode,
   345  				Ts:      row.CommitTs,
   346  				BatchID: 1,
   347  			},
   348  			Size: record.Size,
   349  		}
   350  	} else if row.CommitTs < ra.tableResolvedTsMap[tableID].ResolvedTs.Ts {
   351  		log.Panic("commit ts of redo log regressed",
   352  			zap.Int64("tableID", tableID),
   353  			zap.Uint64("commitTs", row.CommitTs),
   354  			zap.Any("resolvedTs", ra.tableResolvedTsMap[tableID]))
   355  	}
   356  
   357  	ra.appliedLogCount++
   358  	return nil
   359  }
   360  
   361  func (ra *RedoApplier) waitTableFlush(
   362  	ctx context.Context, tableID model.TableID, rts model.Ts,
   363  ) error {
   364  	ticker := time.NewTicker(warnDuration)
   365  	defer ticker.Stop()
   366  
   367  	oldTableRecord := ra.tableResolvedTsMap[tableID]
   368  	if oldTableRecord.ResolvedTs.Ts < rts {
   369  		// Use new batch resolvedTs to flush data.
   370  		ra.tableResolvedTsMap[tableID] = &memquota.MemConsumeRecord{
   371  			ResolvedTs: model.ResolvedTs{
   372  				Mode:    model.BatchResolvedMode,
   373  				Ts:      rts,
   374  				BatchID: 1,
   375  			},
   376  			Size: ra.tableResolvedTsMap[tableID].Size,
   377  		}
   378  	} else if oldTableRecord.ResolvedTs.Ts > rts {
   379  		log.Panic("resolved ts of redo log regressed",
   380  			zap.Any("oldResolvedTs", oldTableRecord),
   381  			zap.Any("newResolvedTs", rts))
   382  	}
   383  
   384  	tableRecord := ra.tableResolvedTsMap[tableID]
   385  	if err := ra.tableSinks[tableID].UpdateResolvedTs(tableRecord.ResolvedTs); err != nil {
   386  		return err
   387  	}
   388  	ra.memQuota.Record(spanz.TableIDToComparableSpan(tableID),
   389  		tableRecord.ResolvedTs, tableRecord.Size)
   390  
   391  	// Make sure all events are flushed to downstream.
   392  	for !ra.tableSinks[tableID].GetCheckpointTs().EqualOrGreater(tableRecord.ResolvedTs) {
   393  		select {
   394  		case <-ctx.Done():
   395  			return errors.Trace(ctx.Err())
   396  		case <-ticker.C:
   397  			log.Warn(
   398  				"Table sink is not catching up with resolved ts for a long time",
   399  				zap.Int64("tableID", tableID),
   400  				zap.Any("resolvedTs", tableRecord.ResolvedTs),
   401  				zap.Any("checkpointTs", ra.tableSinks[tableID].GetCheckpointTs()),
   402  			)
   403  		default:
   404  			time.Sleep(flushWaitDuration)
   405  		}
   406  	}
   407  
   408  	// reset new record
   409  	ra.tableResolvedTsMap[tableID] = &memquota.MemConsumeRecord{
   410  		ResolvedTs: tableRecord.ResolvedTs.AdvanceBatch(),
   411  		Size:       0,
   412  	}
   413  	return nil
   414  }
   415  
   416  var createRedoReader = createRedoReaderImpl
   417  
   418  func createRedoReaderImpl(ctx context.Context, cfg *RedoApplierConfig) (reader.RedoLogReader, error) {
   419  	storageType, readerCfg, err := cfg.toLogReaderConfig()
   420  	if err != nil {
   421  		return nil, err
   422  	}
   423  	return reader.NewRedoLogReader(ctx, storageType, readerCfg)
   424  }
   425  
   426  // tempTxnInsertEventStorage is used to store insert events in the same transaction
   427  // once you begin to read events from storage, you should read all events before you write new events
   428  type tempTxnInsertEventStorage struct {
   429  	events []*model.RowChangedEvent
   430  	// when events num exceed flushThreshold, write all events to file
   431  	flushThreshold int
   432  	dir            string
   433  	txnCommitTs    model.Ts
   434  
   435  	useFileStorage bool
   436  	// eventSizes is used to store the size of each event in file storage
   437  	eventSizes  []int
   438  	writingFile *os.File
   439  	readingFile *os.File
   440  	// reading is used to indicate whether we are reading events from storage
   441  	// this is to ensure that we read all events before write new events
   442  	reading bool
   443  }
   444  
   445  const (
   446  	tempStorageFileName   = "_insert_storage.tmp"
   447  	defaultFlushThreshold = 50
   448  )
   449  
   450  func newTempTxnInsertEventStorage(flushThreshold int, dir string) *tempTxnInsertEventStorage {
   451  	return &tempTxnInsertEventStorage{
   452  		events:         make([]*model.RowChangedEvent, 0),
   453  		flushThreshold: flushThreshold,
   454  		dir:            dir,
   455  		txnCommitTs:    0,
   456  
   457  		useFileStorage: false,
   458  		eventSizes:     make([]int, 0),
   459  
   460  		reading: false,
   461  	}
   462  }
   463  
   464  func (t *tempTxnInsertEventStorage) initializeAddEvent(ts model.Ts) {
   465  	t.reading = false
   466  	t.useFileStorage = false
   467  	t.txnCommitTs = ts
   468  	t.writingFile = nil
   469  	t.readingFile = nil
   470  }
   471  
   472  func (t *tempTxnInsertEventStorage) addEvent(event *model.RowChangedEvent) error {
   473  	// do some pre check
   474  	if !event.IsInsert() {
   475  		log.Panic("event is not insert event", zap.Any("event", event))
   476  	}
   477  	if t.reading && t.hasEvent() {
   478  		log.Panic("should read all events before write new event")
   479  	}
   480  	if !t.hasEvent() {
   481  		t.initializeAddEvent(event.CommitTs)
   482  	} else {
   483  		if t.txnCommitTs != event.CommitTs {
   484  			log.Panic("commit ts of events should be the same",
   485  				zap.Uint64("commitTs", event.CommitTs),
   486  				zap.Uint64("txnCommitTs", t.txnCommitTs))
   487  		}
   488  	}
   489  
   490  	if t.useFileStorage {
   491  		return t.writeEventsToFile(event)
   492  	}
   493  
   494  	t.events = append(t.events, event)
   495  	if len(t.events) >= t.flushThreshold {
   496  		err := t.writeEventsToFile(t.events...)
   497  		if err != nil {
   498  			return err
   499  		}
   500  		t.events = t.events[:0]
   501  	}
   502  	return nil
   503  }
   504  
   505  func (t *tempTxnInsertEventStorage) writeEventsToFile(events ...*model.RowChangedEvent) error {
   506  	if !t.useFileStorage {
   507  		t.useFileStorage = true
   508  		var err error
   509  		t.writingFile, err = os.Create(fmt.Sprintf("%s/%s", t.dir, tempStorageFileName))
   510  		if err != nil {
   511  			return err
   512  		}
   513  	}
   514  	for _, event := range events {
   515  		redoLog := event.ToRedoLog()
   516  		data, err := codec.MarshalRedoLog(redoLog, nil)
   517  		if err != nil {
   518  			return errors.WrapError(errors.ErrMarshalFailed, err)
   519  		}
   520  		t.eventSizes = append(t.eventSizes, len(data))
   521  		_, err = t.writingFile.Write(data)
   522  		if err != nil {
   523  			return err
   524  		}
   525  	}
   526  	return nil
   527  }
   528  
   529  func (t *tempTxnInsertEventStorage) hasEvent() bool {
   530  	return len(t.events) > 0 || len(t.eventSizes) > 0
   531  }
   532  
   533  func (t *tempTxnInsertEventStorage) readFromFile() (*model.RowChangedEvent, error) {
   534  	if len(t.eventSizes) == 0 {
   535  		return nil, nil
   536  	}
   537  	if t.readingFile == nil {
   538  		var err error
   539  		t.readingFile, err = os.Open(fmt.Sprintf("%s/%s", t.dir, tempStorageFileName))
   540  		if err != nil {
   541  			return nil, err
   542  		}
   543  	}
   544  	size := t.eventSizes[0]
   545  	data := make([]byte, size)
   546  	n, err := t.readingFile.Read(data)
   547  	if err != nil {
   548  		return nil, err
   549  	}
   550  	if n != size {
   551  		return nil, errors.New("read size not equal to expected size")
   552  	}
   553  	t.eventSizes = t.eventSizes[1:]
   554  	redoLog, _, err := codec.UnmarshalRedoLog(data)
   555  	if err != nil {
   556  		return nil, errors.WrapError(errors.ErrUnmarshalFailed, err)
   557  	}
   558  	return redoLog.RedoRow.Row.ToRowChangedEvent(), nil
   559  }
   560  
   561  func (t *tempTxnInsertEventStorage) readNextEvent() (*model.RowChangedEvent, error) {
   562  	if !t.hasEvent() {
   563  		return nil, nil
   564  	}
   565  	t.reading = true
   566  	if t.useFileStorage {
   567  		return t.readFromFile()
   568  	}
   569  
   570  	event := t.events[0]
   571  	t.events = t.events[1:]
   572  	return event, nil
   573  }
   574  
   575  // updateEventSplitter splits an update event to a delete event and a deferred insert event
   576  // when the update event is an update to the handle key or the non empty unique key.
   577  // deferred insert event means all delete events and update events in the same transaction are emitted before this insert event
   578  type updateEventSplitter struct {
   579  	rd             reader.RedoLogReader
   580  	rdFinished     bool
   581  	tempStorage    *tempTxnInsertEventStorage
   582  	prevTxnStartTs model.Ts
   583  	// pendingEvent is the event that trigger the process to emit events from tempStorage, it can be
   584  	// 1) an insert event in the same transaction(because there will be no more update and delete events in the same transaction)
   585  	// 2) a new event in the next transaction
   586  	pendingEvent *model.RowChangedEvent
   587  	// meetInsertInCurTxn is used to indicate whether we meet an insert event in the current transaction
   588  	// this is to add some check to ensure that insert events are emitted after other kinds of events in the same transaction
   589  	meetInsertInCurTxn bool
   590  }
   591  
   592  func newUpdateEventSplitter(rd reader.RedoLogReader, dir string) *updateEventSplitter {
   593  	return &updateEventSplitter{
   594  		rd:             rd,
   595  		rdFinished:     false,
   596  		tempStorage:    newTempTxnInsertEventStorage(defaultFlushThreshold, dir),
   597  		prevTxnStartTs: 0,
   598  	}
   599  }
   600  
   601  // processEvent return (event to emit, pending event)
   602  func processEvent(
   603  	event *model.RowChangedEvent,
   604  	prevTxnStartTs model.Ts,
   605  	tempStorage *tempTxnInsertEventStorage,
   606  ) (*model.RowChangedEvent, *model.RowChangedEvent, error) {
   607  	if event == nil {
   608  		log.Panic("event should not be nil")
   609  	}
   610  
   611  	// meet a new transaction
   612  	if prevTxnStartTs != 0 && prevTxnStartTs != event.StartTs {
   613  		if tempStorage.hasEvent() {
   614  			// emit the insert events in the previous transaction
   615  			return nil, event, nil
   616  		}
   617  	}
   618  	if event.IsDelete() {
   619  		return event, nil, nil
   620  	} else if event.IsInsert() {
   621  		if tempStorage.hasEvent() {
   622  			// pend current event and emit the insert events in temp storage first to release memory
   623  			return nil, event, nil
   624  		}
   625  		return event, nil, nil
   626  	} else if !model.ShouldSplitUpdateEvent(event) {
   627  		return event, nil, nil
   628  	} else {
   629  		deleteEvent, insertEvent, err := model.SplitUpdateEvent(event)
   630  		if err != nil {
   631  			return nil, nil, err
   632  		}
   633  		err = tempStorage.addEvent(insertEvent)
   634  		if err != nil {
   635  			return nil, nil, err
   636  		}
   637  		return deleteEvent, nil, nil
   638  	}
   639  }
   640  
   641  func (u *updateEventSplitter) checkEventOrder(event *model.RowChangedEvent) {
   642  	if event == nil {
   643  		return
   644  	}
   645  	// meeet a new transaction
   646  	if event.StartTs != u.prevTxnStartTs {
   647  		u.meetInsertInCurTxn = false
   648  		return
   649  	}
   650  	if event.IsInsert() {
   651  		u.meetInsertInCurTxn = true
   652  	} else {
   653  		// delete or update events
   654  		if u.meetInsertInCurTxn {
   655  			log.Panic("insert events should be emitted after other kinds of events in the same transaction")
   656  		}
   657  	}
   658  }
   659  
   660  func (u *updateEventSplitter) readNextRow(ctx context.Context) (*model.RowChangedEvent, error) {
   661  	for {
   662  		// case 1: pendingEvent is not nil, emit all events from tempStorage and then process pendingEvent
   663  		if u.pendingEvent != nil {
   664  			if u.tempStorage.hasEvent() {
   665  				return u.tempStorage.readNextEvent()
   666  			}
   667  			var event *model.RowChangedEvent
   668  			var err error
   669  			event, u.pendingEvent, err = processEvent(u.pendingEvent, u.prevTxnStartTs, u.tempStorage)
   670  			if err != nil {
   671  				return nil, err
   672  			}
   673  			if event == nil || u.pendingEvent != nil {
   674  				log.Panic("processEvent return wrong result for pending event",
   675  					zap.Any("event", event),
   676  					zap.Any("pendingEvent", u.pendingEvent))
   677  			}
   678  			return event, nil
   679  		}
   680  		// case 2: no more events from RedoLogReader, emit all events from tempStorage and return nil
   681  		if u.rdFinished {
   682  			if u.tempStorage.hasEvent() {
   683  				return u.tempStorage.readNextEvent()
   684  			}
   685  			return nil, nil
   686  		}
   687  		// case 3: read and process events from RedoLogReader
   688  		event, err := u.rd.ReadNextRow(ctx)
   689  		if err != nil {
   690  			return nil, err
   691  		}
   692  		if event == nil {
   693  			u.rdFinished = true
   694  		} else {
   695  			u.checkEventOrder(event)
   696  			prevTxnStartTs := u.prevTxnStartTs
   697  			u.prevTxnStartTs = event.StartTs
   698  			var err error
   699  			event, u.pendingEvent, err = processEvent(event, prevTxnStartTs, u.tempStorage)
   700  			if err != nil {
   701  				return nil, err
   702  			}
   703  			if event != nil {
   704  				return event, nil
   705  			}
   706  			if u.pendingEvent == nil {
   707  				log.Panic("event to emit and pending event cannot all be nil")
   708  			}
   709  		}
   710  	}
   711  }
   712  
   713  // ReadMeta creates a new redo applier and read meta from reader
   714  func (ra *RedoApplier) ReadMeta(ctx context.Context) (checkpointTs uint64, resolvedTs uint64, err error) {
   715  	rd, err := createRedoReader(ctx, ra.cfg)
   716  	if err != nil {
   717  		return 0, 0, err
   718  	}
   719  	return rd.ReadMeta(ctx)
   720  }
   721  
   722  // Apply applies redo log to given target
   723  func (ra *RedoApplier) Apply(egCtx context.Context) (err error) {
   724  	eg, egCtx := errgroup.WithContext(egCtx)
   725  
   726  	if ra.rd, err = createRedoReader(egCtx, ra.cfg); err != nil {
   727  		return err
   728  	}
   729  	eg.Go(func() error {
   730  		return ra.rd.Run(egCtx)
   731  	})
   732  	ra.updateSplitter = newUpdateEventSplitter(ra.rd, ra.cfg.Dir)
   733  
   734  	ra.memQuota = memquota.NewMemQuota(model.DefaultChangeFeedID(applierChangefeed),
   735  		config.DefaultChangefeedMemoryQuota, "sink")
   736  	defer ra.memQuota.Close()
   737  	eg.Go(func() error {
   738  		return ra.bgReleaseQuota(egCtx)
   739  	})
   740  
   741  	eg.Go(func() error {
   742  		return ra.consumeLogs(egCtx)
   743  	})
   744  	eg.Go(func() error {
   745  		return ra.catchError(egCtx)
   746  	})
   747  
   748  	err = eg.Wait()
   749  	if errors.Cause(err) != errApplyFinished {
   750  		return err
   751  	}
   752  	return nil
   753  }