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

     1  // Copyright 2023 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 owner
    15  
    16  import (
    17  	"context"
    18  	"math/rand"
    19  	"sort"
    20  	"time"
    21  
    22  	"github.com/pingcap/errors"
    23  	"github.com/pingcap/failpoint"
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/tidb/pkg/parser/ast"
    26  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    27  	"github.com/pingcap/tiflow/cdc/entry"
    28  	"github.com/pingcap/tiflow/cdc/model"
    29  	"github.com/pingcap/tiflow/cdc/puller"
    30  	"github.com/pingcap/tiflow/cdc/redo"
    31  	"github.com/pingcap/tiflow/cdc/scheduler/schedulepb"
    32  	"github.com/pingcap/tiflow/pkg/filter"
    33  	"go.uber.org/zap"
    34  )
    35  
    36  // tableBarrierNumberLimit is used to limit the number
    37  // of tableBarrier in a single barrier.
    38  const tableBarrierNumberLimit = 256
    39  
    40  // The ddls below is globalDDLs, they affect all tables in the changefeed.
    41  // we need to wait all tables checkpointTs reach the DDL commitTs
    42  // before we can execute the DDL.
    43  //timodel.ActionCreateSchema
    44  //timodel.ActionDropSchema
    45  //timodel.ActionModifySchemaCharsetAndCollate
    46  //// We treat create table ddl as a global ddl, because before we execute the ddl,
    47  //// there is no a tablePipeline for the new table. So we can't prevent the checkpointTs
    48  //// from advancing. To solve this problem, we just treat create table ddl as a global ddl here.
    49  //// TODO: Find a better way to handle create table ddl.
    50  //timodel.ActionCreateTable
    51  //timodel.ActionRenameTable
    52  //timodel.ActionRenameTables
    53  //timodel.ActionExchangeTablePartition
    54  
    55  // nonGlobalDDLs are the DDLs that only affect related table
    56  // so that we should only block related table before execute them.
    57  var nonGlobalDDLs = map[timodel.ActionType]struct{}{
    58  	timodel.ActionDropTable:                    {},
    59  	timodel.ActionAddColumn:                    {},
    60  	timodel.ActionDropColumn:                   {},
    61  	timodel.ActionAddIndex:                     {},
    62  	timodel.ActionDropIndex:                    {},
    63  	timodel.ActionTruncateTable:                {},
    64  	timodel.ActionModifyColumn:                 {},
    65  	timodel.ActionSetDefaultValue:              {},
    66  	timodel.ActionModifyTableComment:           {},
    67  	timodel.ActionRenameIndex:                  {},
    68  	timodel.ActionAddTablePartition:            {},
    69  	timodel.ActionDropTablePartition:           {},
    70  	timodel.ActionCreateView:                   {},
    71  	timodel.ActionModifyTableCharsetAndCollate: {},
    72  	timodel.ActionTruncateTablePartition:       {},
    73  	timodel.ActionDropView:                     {},
    74  	timodel.ActionRecoverTable:                 {},
    75  	timodel.ActionAddPrimaryKey:                {},
    76  	timodel.ActionDropPrimaryKey:               {},
    77  	timodel.ActionRebaseAutoID:                 {},
    78  	timodel.ActionAlterIndexVisibility:         {},
    79  	timodel.ActionMultiSchemaChange:            {},
    80  	timodel.ActionReorganizePartition:          {},
    81  	timodel.ActionAlterTTLInfo:                 {},
    82  	timodel.ActionAlterTTLRemove:               {},
    83  	timodel.ActionAlterTablePartitioning:       {},
    84  	timodel.ActionRemovePartitioning:           {},
    85  }
    86  
    87  var redoBarrierDDLs = map[timodel.ActionType]struct{}{
    88  	timodel.ActionCreateTable:            {},
    89  	timodel.ActionTruncateTable:          {},
    90  	timodel.ActionAddTablePartition:      {},
    91  	timodel.ActionTruncateTablePartition: {},
    92  	timodel.ActionRecoverTable:           {},
    93  	timodel.ActionReorganizePartition:    {},
    94  	timodel.ActionAlterTablePartitioning: {},
    95  	timodel.ActionRemovePartitioning:     {},
    96  }
    97  
    98  // ddlManager holds the pending DDL events of all tables and responsible for
    99  // executing them to downstream.
   100  // It also provides the ability to calculate the barrier of a changefeed.
   101  type ddlManager struct {
   102  	changfeedID  model.ChangeFeedID
   103  	startTs      model.Ts
   104  	checkpointTs model.Ts
   105  	// use to pull DDL jobs from TiDB
   106  	ddlPuller puller.DDLPuller
   107  	// schema store multiple version of schema, it is used by scheduler
   108  	schema entry.SchemaStorage
   109  	// redoDDLManager is used to send DDL events to redo log and get redo resolvedTs.
   110  	redoDDLManager  redo.DDLManager
   111  	redoMetaManager redo.MetaManager
   112  	// ddlSink is used to ddlSink DDL events to the downstream
   113  	ddlSink DDLSink
   114  	filter  filter.Filter
   115  
   116  	// pendingDDLs store the pending DDL events of all tables
   117  	// the DDL events in the same table are ordered by commitTs.
   118  	pendingDDLs map[model.TableName][]*model.DDLEvent
   119  	// executingDDL is the ddl that is currently being executed,
   120  	// it is nil if there is no ddl being executed.
   121  	executingDDL *model.DDLEvent
   122  	// justSentDDL is the ddl that just be sent to the downstream in the current tick.
   123  	// we need it to prevent the checkpointTs from advancing in the same tick.
   124  	justSentDDL *model.DDLEvent
   125  	// tableInfoCache is the tables that the changefeed is watching.
   126  	// And it contains only the tables of the ddl that have been processed.
   127  	// The ones that have not been executed yet do not have.
   128  	tableInfoCache      []*model.TableInfo
   129  	physicalTablesCache []model.TableID
   130  
   131  	BDRMode       bool
   132  	ddlResolvedTs model.Ts
   133  }
   134  
   135  func newDDLManager(
   136  	changefeedID model.ChangeFeedID,
   137  	startTs model.Ts,
   138  	checkpointTs model.Ts,
   139  	ddlSink DDLSink,
   140  	filter filter.Filter,
   141  	ddlPuller puller.DDLPuller,
   142  	schema entry.SchemaStorage,
   143  	redoManager redo.DDLManager,
   144  	redoMetaManager redo.MetaManager,
   145  	bdrMode bool,
   146  ) *ddlManager {
   147  	log.Info("owner create ddl manager",
   148  		zap.String("namespace", changefeedID.Namespace),
   149  		zap.String("changefeed", changefeedID.ID),
   150  		zap.Uint64("startTs", startTs),
   151  		zap.Uint64("checkpointTs", checkpointTs),
   152  		zap.Bool("bdrMode", bdrMode))
   153  
   154  	return &ddlManager{
   155  		changfeedID:     changefeedID,
   156  		ddlSink:         ddlSink,
   157  		filter:          filter,
   158  		ddlPuller:       ddlPuller,
   159  		schema:          schema,
   160  		redoDDLManager:  redoManager,
   161  		redoMetaManager: redoMetaManager,
   162  		startTs:         startTs,
   163  		checkpointTs:    checkpointTs,
   164  		ddlResolvedTs:   startTs,
   165  		BDRMode:         bdrMode,
   166  		pendingDDLs:     make(map[model.TableName][]*model.DDLEvent),
   167  	}
   168  }
   169  
   170  // tick the ddlHandler, it does the following things:
   171  // 1. get DDL jobs from ddlPuller.
   172  // 2. uses schema to turn DDL jobs into DDLEvents.
   173  // 3. applies DDL jobs to the schema.
   174  // 4. send DDLEvents to redo log.
   175  // 5. adds the DDLEvents to the ddlHandler.pendingDDLs
   176  // 6. iterates the ddlHandler.pendingDDLs, find next DDL event to be executed.
   177  // 7.checks if checkpointTs reach next ddl commitTs, if so, execute the ddl.
   178  // 8. removes the executed DDL events from executingDDL and pendingDDLs.
   179  func (m *ddlManager) tick(
   180  	ctx context.Context,
   181  	checkpointTs model.Ts,
   182  ) ([]model.TableID, *schedulepb.BarrierWithMinTs, error) {
   183  	m.justSentDDL = nil
   184  	m.checkpointTs = checkpointTs
   185  
   186  	currentTables, err := m.allTables(ctx)
   187  	if err != nil {
   188  		return nil, nil, errors.Trace(err)
   189  	}
   190  
   191  	if m.executingDDL == nil {
   192  		m.ddlSink.emitCheckpointTs(m.checkpointTs, currentTables)
   193  	}
   194  
   195  	tableIDs, err := m.allPhysicalTables(ctx)
   196  	if err != nil {
   197  		return nil, nil, errors.Trace(err)
   198  	}
   199  
   200  	// drain all ddl jobs from ddlPuller
   201  	for {
   202  		_, job := m.ddlPuller.PopFrontDDL()
   203  		// no more ddl jobs
   204  		if job == nil {
   205  			break
   206  		}
   207  
   208  		if job.BinlogInfo == nil {
   209  			continue
   210  		}
   211  
   212  		log.Info("handle a ddl job",
   213  			zap.String("namespace", m.changfeedID.Namespace),
   214  			zap.String("changefeed", m.changfeedID.ID),
   215  			zap.Int64("tableID", job.TableID),
   216  			zap.Int64("jobID", job.ID),
   217  			zap.String("query", job.Query),
   218  			zap.Uint64("finishedTs", job.BinlogInfo.FinishedTS),
   219  		)
   220  		events, err := m.schema.BuildDDLEvents(ctx, job)
   221  		if err != nil {
   222  			return nil, nil, err
   223  		}
   224  
   225  		for _, event := range events {
   226  			tableName := event.TableInfo.TableName
   227  			m.pendingDDLs[tableName] = append(m.pendingDDLs[tableName], event)
   228  		}
   229  
   230  		// Send DDL events to redo log.
   231  		if m.redoDDLManager.Enabled() {
   232  			for _, event := range events {
   233  				skip, _, err := m.shouldSkipDDL(event)
   234  				if err != nil {
   235  					return nil, nil, errors.Trace(err)
   236  				}
   237  				if skip {
   238  					continue
   239  				}
   240  				if err := m.redoDDLManager.EmitDDLEvent(ctx, event); err != nil {
   241  					return nil, nil, err
   242  				}
   243  			}
   244  		}
   245  	}
   246  
   247  	// advance resolvedTs
   248  	ddlRts := m.ddlPuller.ResolvedTs()
   249  	m.schema.AdvanceResolvedTs(ddlRts)
   250  	if m.redoDDLManager.Enabled() {
   251  		err := m.redoDDLManager.UpdateResolvedTs(ctx, ddlRts)
   252  		if err != nil {
   253  			return nil, nil, err
   254  		}
   255  		redoFlushedDDLRts := m.redoDDLManager.GetResolvedTs()
   256  		if redoFlushedDDLRts < ddlRts {
   257  			ddlRts = redoFlushedDDLRts
   258  		}
   259  	}
   260  	if m.ddlResolvedTs <= ddlRts {
   261  		m.ddlResolvedTs = ddlRts
   262  	}
   263  
   264  	nextDDL := m.getNextDDL()
   265  	if nextDDL != nil {
   266  		if m.checkpointTs > nextDDL.CommitTs {
   267  			log.Panic("checkpointTs is greater than next ddl commitTs",
   268  				zap.Uint64("checkpointTs", m.checkpointTs),
   269  				zap.Uint64("commitTs", nextDDL.CommitTs))
   270  		}
   271  
   272  		if m.shouldExecDDL(nextDDL) {
   273  			if m.executingDDL == nil {
   274  				log.Info("execute a ddl event",
   275  					zap.String("query", nextDDL.Query),
   276  					zap.Uint64("commitTs", nextDDL.CommitTs),
   277  					zap.Uint64("checkpointTs", m.checkpointTs))
   278  				m.executingDDL = nextDDL
   279  			}
   280  			err := m.executeDDL(ctx)
   281  			if err != nil {
   282  				return nil, nil, err
   283  			}
   284  		}
   285  	}
   286  
   287  	return tableIDs, m.barrier(), nil
   288  }
   289  
   290  func (m *ddlManager) shouldExecDDL(nextDDL *model.DDLEvent) bool {
   291  	// TiCDC guarantees all dml(s) that happen before a ddl was sent to
   292  	// downstream when this ddl is sent. So, we need to wait checkpointTs is
   293  	// fullyBlocked at ddl commitTs (equivalent to ddl commitTs here) before we
   294  	// execute the next ddl.
   295  	// For example, let say there are some events are replicated by cdc:
   296  	// [dml-1(ts=5), dml-2(ts=8), dml-3(ts=11), ddl-1(ts=11), ddl-2(ts=12)].
   297  	// We need to wait `checkpointTs == ddlCommitTs(ts=11)` before executing ddl-1.
   298  	checkpointReachBarrier := m.checkpointTs == nextDDL.CommitTs
   299  
   300  	redoCheckpointReachBarrier := true
   301  	redoDDLResolvedTsExceedBarrier := true
   302  	if m.redoMetaManager.Enabled() {
   303  		if !m.redoDDLManager.Enabled() {
   304  			log.Panic("Redo meta manager is enabled but redo ddl manager is not enabled")
   305  		}
   306  		flushed := m.redoMetaManager.GetFlushedMeta()
   307  		// Use the same example as above, let say there are some events are replicated by cdc:
   308  		// [dml-1(ts=5), dml-2(ts=8), dml-3(ts=11), ddl-1(ts=11), ddl-2(ts=12)].
   309  		// Suppose redoCheckpointTs=10 and ddl-1(ts=11) is executed, the redo apply operation
   310  		// would fail when applying the old data dml-3(ts=11) to a new schmea. Therefore, We
   311  		// need to wait `redoCheckpointTs == ddlCommitTs(ts=11)` before executing ddl-1.
   312  		redoCheckpointReachBarrier = flushed.CheckpointTs == nextDDL.CommitTs
   313  
   314  		// If redo is enabled, m.ddlResolvedTs == redoDDLManager.GetResolvedTs(), so we need to
   315  		// wait nextDDL to be written to redo log before executing this DDL.
   316  		redoDDLResolvedTsExceedBarrier = m.ddlResolvedTs >= nextDDL.CommitTs
   317  	}
   318  
   319  	return checkpointReachBarrier && redoCheckpointReachBarrier && redoDDLResolvedTsExceedBarrier
   320  }
   321  
   322  func (m *ddlManager) shouldSkipDDL(ddl *model.DDLEvent) (bool, string, error) {
   323  	ignored, err := m.filter.ShouldIgnoreDDLEvent(ddl)
   324  	if err != nil {
   325  		return false, "", errors.Trace(err)
   326  	}
   327  	if ignored {
   328  		return true, "ddl is ignored by event filter rule, skip it", nil
   329  	}
   330  
   331  	// In a BDR mode cluster, TiCDC can receive DDLs from all roles of TiDB.
   332  	// However, CDC only executes the DDLs from the TiDB that has BDRRolePrimary role.
   333  	if m.BDRMode && ddl.BDRRole != string(ast.BDRRolePrimary) {
   334  		return true, "changefeed is in BDRMode and the DDL is not executed by Primary Cluster, skip it", nil
   335  	}
   336  	return false, "", nil
   337  }
   338  
   339  // executeDDL executes ddlManager.executingDDL.
   340  func (m *ddlManager) executeDDL(ctx context.Context) error {
   341  	if m.executingDDL == nil {
   342  		return nil
   343  	}
   344  	skip, cleanMsg, err := m.shouldSkipDDL(m.executingDDL)
   345  	if err != nil {
   346  		return errors.Trace(err)
   347  	}
   348  	if skip {
   349  		m.cleanCache(cleanMsg)
   350  		return nil
   351  	}
   352  
   353  	failpoint.Inject("ExecuteNotDone", func() {
   354  		// This ddl will never finish executing.
   355  		// It is used to test the logic that a ddl only block the related table
   356  		// and other tables can still advance.
   357  		if m.executingDDL.TableInfo.TableName.Table == "ddl_not_done" {
   358  			time.Sleep(time.Second * 1)
   359  			failpoint.Return(nil)
   360  		}
   361  	})
   362  
   363  	failpoint.Inject("ExecuteDDLSlowly", func() {
   364  		lag := time.Duration(rand.Intn(5000)) * time.Millisecond
   365  		log.Warn("execute ddl slowly", zap.Duration("lag", lag))
   366  		time.Sleep(lag)
   367  	})
   368  
   369  	done, err := m.ddlSink.emitDDLEvent(ctx, m.executingDDL)
   370  	if err != nil {
   371  		return errors.Trace(err)
   372  	}
   373  	if done {
   374  		m.cleanCache("execute a ddl event successfully")
   375  	}
   376  	return nil
   377  }
   378  
   379  // getNextDDL returns the next ddl event to execute.
   380  func (m *ddlManager) getNextDDL() *model.DDLEvent {
   381  	if m.executingDDL != nil {
   382  		return m.executingDDL
   383  	}
   384  	var res *model.DDLEvent
   385  	for tb, ddls := range m.pendingDDLs {
   386  		if len(ddls) == 0 {
   387  			log.Debug("no more ddl event, gc the table from pendingDDLs",
   388  				zap.String("table", tb.String()))
   389  			delete(m.pendingDDLs, tb)
   390  			continue
   391  		}
   392  		if res == nil || res.CommitTs > ddls[0].CommitTs {
   393  			res = ddls[0]
   394  		}
   395  	}
   396  	return res
   397  }
   398  
   399  // getAllTableNextDDL returns the next DDL of all tables.
   400  func (m *ddlManager) getAllTableNextDDL() []*model.DDLEvent {
   401  	res := make([]*model.DDLEvent, 0, 1)
   402  	for _, events := range m.pendingDDLs {
   403  		if len(events) > 0 {
   404  			res = append(res, events[0])
   405  		}
   406  	}
   407  	return res
   408  }
   409  
   410  // barrier returns ddlResolvedTs and tableBarrier
   411  func (m *ddlManager) barrier() *schedulepb.BarrierWithMinTs {
   412  	barrier := schedulepb.NewBarrierWithMinTs(m.ddlResolvedTs)
   413  	tableBarrierMap := make(map[model.TableID]model.Ts)
   414  	ddls := m.getAllTableNextDDL()
   415  	if m.justSentDDL != nil {
   416  		ddls = append(ddls, m.justSentDDL)
   417  	}
   418  
   419  	for _, ddl := range ddls {
   420  		if ddl.CommitTs < barrier.MinTableBarrierTs {
   421  			barrier.MinTableBarrierTs = ddl.CommitTs
   422  		}
   423  		if m.redoMetaManager.Enabled() && isRedoBarrierDDL(ddl) {
   424  			// The pipeline for a new table does not exist until the ddl is successfully
   425  			// executed, so the table's resolvedTs will not be calculated in redo.
   426  			// To solve this problem, resovedTs of redo manager should not be greater
   427  			// than the min commitTs of ddls that create a new physical table.
   428  			if ddl.CommitTs < barrier.RedoBarrierTs {
   429  				barrier.RedoBarrierTs = ddl.CommitTs
   430  			}
   431  		}
   432  		if isGlobalDDL(ddl) {
   433  			// When there is a global DDL, we need to wait all tables
   434  			// checkpointTs reach its commitTs before we can execute it.
   435  			if ddl.CommitTs < barrier.GlobalBarrierTs {
   436  				barrier.GlobalBarrierTs = ddl.CommitTs
   437  			}
   438  		} else {
   439  			// barrier related physical tables
   440  			ids := getRelatedPhysicalTableIDs(ddl)
   441  			for _, id := range ids {
   442  				// The same physical table may have multiple related ddl events when calculating barrier.
   443  				// Example cases:
   444  				// 1. The logical id of the same partition table may change after change partition.
   445  				//	So the related ddls may be considered for different tables.
   446  				//  And they may be returned by `getAllTableNextDDL` at the same time.
   447  				// 2. The result of `getAllTableNextDDL` may influence the same physical tables as `ddlManager.justSentDDL`.
   448  				// So we always choose the min commitTs of all ddls related to the same physical table as the barrierTs.
   449  				if ts, ok := tableBarrierMap[id]; ok {
   450  					if ddl.CommitTs < ts {
   451  						tableBarrierMap[id] = ddl.CommitTs
   452  					}
   453  				} else {
   454  					tableBarrierMap[id] = ddl.CommitTs
   455  				}
   456  			}
   457  		}
   458  	}
   459  
   460  	// calculate tableBarriers
   461  	var tableBarriers []*schedulepb.TableBarrier
   462  	for tableID, tableBarrierTs := range tableBarrierMap {
   463  		if tableBarrierTs > barrier.GlobalBarrierTs {
   464  			continue
   465  		}
   466  		tableBarriers = append(tableBarriers, &schedulepb.TableBarrier{
   467  			TableID:   tableID,
   468  			BarrierTs: tableBarrierTs,
   469  		})
   470  	}
   471  	// Limit the tableBarrier size to avoid too large barrier. Since it will
   472  	// cause the scheduler to be slow.
   473  	sort.Slice(tableBarriers, func(i, j int) bool {
   474  		return tableBarriers[i].BarrierTs < tableBarriers[j].BarrierTs
   475  	})
   476  	if len(tableBarriers) > tableBarrierNumberLimit {
   477  		barrier.GlobalBarrierTs = tableBarriers[tableBarrierNumberLimit].BarrierTs
   478  		tableBarriers = tableBarriers[:tableBarrierNumberLimit]
   479  	}
   480  
   481  	m.justSentDDL = nil
   482  	barrier.TableBarriers = tableBarriers
   483  	return barrier
   484  }
   485  
   486  // allTables returns all tables in the schema that
   487  // less or equal than the checkpointTs.
   488  func (m *ddlManager) allTables(ctx context.Context) ([]*model.TableInfo, error) {
   489  	if m.tableInfoCache == nil {
   490  		ts := m.getSnapshotTs()
   491  		tableInfoCache, err := m.schema.AllTables(ctx, ts)
   492  		if err != nil {
   493  			return nil, err
   494  		}
   495  		m.tableInfoCache = tableInfoCache
   496  		log.Debug("changefeed current tables updated",
   497  			zap.String("namespace", m.changfeedID.Namespace),
   498  			zap.String("changefeed", m.changfeedID.ID),
   499  			zap.Uint64("checkpointTs", m.checkpointTs),
   500  			zap.Uint64("snapshotTs", ts),
   501  			zap.Any("tables", m.tableInfoCache),
   502  		)
   503  	}
   504  
   505  	return m.tableInfoCache, nil
   506  }
   507  
   508  // allPhysicalTables returns all table ids in the schema
   509  // that less or equal than the checkpointTs.
   510  func (m *ddlManager) allPhysicalTables(ctx context.Context) ([]model.TableID, error) {
   511  	if m.physicalTablesCache == nil {
   512  		ts := m.getSnapshotTs()
   513  		cache, err := m.schema.AllPhysicalTables(ctx, ts)
   514  		if err != nil {
   515  			return nil, err
   516  		}
   517  		log.Debug("changefeed physical tables updated",
   518  			zap.String("namespace", m.changfeedID.Namespace),
   519  			zap.String("changefeed", m.changfeedID.ID),
   520  			zap.Uint64("checkpointTs", m.checkpointTs),
   521  			zap.Uint64("snapshotTs", ts),
   522  			zap.Any("tables", m.physicalTablesCache),
   523  		)
   524  		m.physicalTablesCache = cache
   525  	}
   526  	return m.physicalTablesCache, nil
   527  }
   528  
   529  // getSnapshotTs returns the ts that we should use
   530  // to get the snapshot of the schema, the rules are:
   531  // If the changefeed is just started, we use the startTs,
   532  // otherwise we use the checkpointTs.
   533  func (m *ddlManager) getSnapshotTs() (ts uint64) {
   534  	ts = m.checkpointTs
   535  
   536  	if m.ddlResolvedTs == m.startTs {
   537  		// If ddlResolvedTs is equal to startTs it means that the changefeed is just started,
   538  		// So we need to get all tables from the snapshot at the startTs.
   539  		ts = m.startTs
   540  		log.Debug("changefeed is just started, use startTs to get snapshot",
   541  			zap.String("namespace", m.changfeedID.Namespace),
   542  			zap.String("changefeed", m.changfeedID.ID),
   543  			zap.Uint64("startTs", m.startTs),
   544  			zap.Uint64("checkpointTs", m.checkpointTs),
   545  			zap.Uint64("ddlResolvedTs", m.ddlResolvedTs),
   546  		)
   547  		return
   548  	}
   549  
   550  	log.Debug("snapshotTs", zap.Uint64("ts", ts))
   551  	return ts
   552  }
   553  
   554  // cleanCache cleans the tableInfoCache and physicalTablesCache.
   555  // It should be called after a DDL is skipped or sent to downstream successfully.
   556  func (m *ddlManager) cleanCache(msg string) {
   557  	tableName := m.executingDDL.TableInfo.TableName
   558  	log.Info(msg, zap.String("ddl", m.executingDDL.Query),
   559  		zap.String("namespace", m.changfeedID.Namespace),
   560  		zap.String("changefeed", m.changfeedID.ID),
   561  		zap.String("bdrRole", m.executingDDL.BDRRole),
   562  		zap.Any("ddlEvent", m.executingDDL))
   563  
   564  	// Set it to nil first to accelerate GC.
   565  	m.pendingDDLs[tableName][0] = nil
   566  	m.pendingDDLs[tableName] = m.pendingDDLs[tableName][1:]
   567  	m.schema.DoGC(m.executingDDL.CommitTs - 1)
   568  	m.justSentDDL = m.executingDDL
   569  	m.executingDDL = nil
   570  
   571  	m.tableInfoCache = nil
   572  	m.physicalTablesCache = nil
   573  }
   574  
   575  // getRelatedPhysicalTableIDs get all related physical table ids of a ddl event.
   576  // It is a helper function to calculate tableBarrier.
   577  func getRelatedPhysicalTableIDs(ddl *model.DDLEvent) []model.TableID {
   578  	res := make([]model.TableID, 0, 1)
   579  	table := ddl.TableInfo
   580  	if ddl.PreTableInfo != nil {
   581  		table = ddl.PreTableInfo
   582  	}
   583  	if table == nil {
   584  		// If the table is nil, it means that the ddl is a global ddl.
   585  		// It should never go here.
   586  		log.Panic("tableInfo of this ddl is nil", zap.Any("ddl", ddl))
   587  	}
   588  	res = append(res, table.ID)
   589  	partitionInfo := table.TableInfo.GetPartitionInfo()
   590  	if partitionInfo != nil {
   591  		for _, def := range partitionInfo.Definitions {
   592  			res = append(res, def.ID)
   593  		}
   594  	}
   595  	return res
   596  }
   597  
   598  // isGlobalDDL returns whether the ddl is a global ddl.
   599  func isGlobalDDL(ddl *model.DDLEvent) bool {
   600  	_, ok := nonGlobalDDLs[ddl.Type]
   601  	return !ok
   602  }
   603  
   604  func isRedoBarrierDDL(ddl *model.DDLEvent) bool {
   605  	_, ok := redoBarrierDDLs[ddl.Type]
   606  	return ok
   607  }