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

     1  // Copyright 2022 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 puller
    15  
    16  import (
    17  	"context"
    18  	"encoding/json"
    19  	"fmt"
    20  	"sync"
    21  	"sync/atomic"
    22  	"time"
    23  
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/log"
    26  	tidbkv "github.com/pingcap/tidb/pkg/kv"
    27  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    28  	"github.com/pingcap/tidb/pkg/parser/mysql"
    29  	"github.com/pingcap/tiflow/cdc/entry"
    30  	"github.com/pingcap/tiflow/cdc/entry/schema"
    31  	"github.com/pingcap/tiflow/cdc/kv"
    32  	"github.com/pingcap/tiflow/cdc/kv/sharedconn"
    33  	"github.com/pingcap/tiflow/cdc/model"
    34  	"github.com/pingcap/tiflow/cdc/processor/tablepb"
    35  	"github.com/pingcap/tiflow/cdc/puller/memorysorter"
    36  	"github.com/pingcap/tiflow/engine/pkg/clock"
    37  	"github.com/pingcap/tiflow/pkg/config"
    38  	cerror "github.com/pingcap/tiflow/pkg/errors"
    39  	"github.com/pingcap/tiflow/pkg/filter"
    40  	"github.com/pingcap/tiflow/pkg/spanz"
    41  	"github.com/pingcap/tiflow/pkg/txnutil"
    42  	"github.com/pingcap/tiflow/pkg/upstream"
    43  	"github.com/pingcap/tiflow/pkg/util"
    44  	"github.com/tikv/client-go/v2/tikv"
    45  	"go.uber.org/zap"
    46  	"golang.org/x/sync/errgroup"
    47  )
    48  
    49  const (
    50  	ddlPullerStuckWarnDuration = 30 * time.Second
    51  	// ddl puller should never filter any DDL jobs even if
    52  	// the changefeed is in BDR mode, because the DDL jobs should
    53  	// be filtered before they are sent to the sink
    54  	ddlPullerFilterLoop = false
    55  )
    56  
    57  // DDLJobPuller is used to pull ddl job from TiKV.
    58  // It's used by processor and ddlPullerImpl.
    59  type DDLJobPuller interface {
    60  	util.Runnable
    61  
    62  	// Output the DDL job entry, it contains the DDL job and the error.
    63  	Output() <-chan *model.DDLJobEntry
    64  }
    65  
    66  // Note: All unexported methods of `ddlJobPullerImpl` should
    67  // be called in the same one goroutine.
    68  type ddlJobPullerImpl struct {
    69  	changefeedID model.ChangeFeedID
    70  	mp           *MultiplexingPuller
    71  	// memorysorter is used to sort the DDL events.
    72  	sorter        *memorysorter.EntrySorter
    73  	kvStorage     tidbkv.Storage
    74  	schemaStorage entry.SchemaStorage
    75  	resolvedTs    uint64
    76  	schemaVersion int64
    77  	filter        filter.Filter
    78  	// ddlTableInfo is initialized when receive the first concurrent DDL job.
    79  	ddlTableInfo *entry.DDLTableInfo
    80  	// outputCh sends the DDL job entries to the caller.
    81  	outputCh chan *model.DDLJobEntry
    82  }
    83  
    84  // NewDDLJobPuller creates a new NewDDLJobPuller,
    85  // which fetches ddl events starting from checkpointTs.
    86  func NewDDLJobPuller(
    87  	ctx context.Context,
    88  	up *upstream.Upstream,
    89  	checkpointTs uint64,
    90  	cfg *config.ServerConfig,
    91  	changefeed model.ChangeFeedID,
    92  	schemaStorage entry.SchemaStorage,
    93  	filter filter.Filter,
    94  ) DDLJobPuller {
    95  	pdCli := up.PDClient
    96  	regionCache := up.RegionCache
    97  	kvStorage := up.KVStorage
    98  	pdClock := up.PDClock
    99  
   100  	ddlSpans := spanz.GetAllDDLSpan()
   101  	for i := range ddlSpans {
   102  		// NOTE(qupeng): It's better to use different table id when use sharedKvClient.
   103  		ddlSpans[i].TableID = int64(-1) - int64(i)
   104  	}
   105  
   106  	ddlJobPuller := &ddlJobPullerImpl{
   107  		changefeedID:  changefeed,
   108  		schemaStorage: schemaStorage,
   109  		kvStorage:     kvStorage,
   110  		filter:        filter,
   111  		outputCh:      make(chan *model.DDLJobEntry, defaultPullerOutputChanSize),
   112  	}
   113  	ddlJobPuller.sorter = memorysorter.NewEntrySorter(changefeed)
   114  
   115  	grpcPool := sharedconn.NewConnAndClientPool(up.SecurityConfig, kv.GetGlobalGrpcMetrics())
   116  	client := kv.NewSharedClient(
   117  		changefeed, cfg, ddlPullerFilterLoop,
   118  		pdCli, grpcPool, regionCache, pdClock,
   119  		txnutil.NewLockerResolver(kvStorage.(tikv.Storage), changefeed),
   120  	)
   121  
   122  	slots, hasher := 1, func(tablepb.Span, int) int { return 0 }
   123  	ddlJobPuller.mp = NewMultiplexingPuller(changefeed, client, up.PDClock, ddlJobPuller.Input, slots, hasher, 1)
   124  	ddlJobPuller.mp.Subscribe(ddlSpans, checkpointTs, memorysorter.DDLPullerTableName)
   125  
   126  	return ddlJobPuller
   127  }
   128  
   129  // Run implements util.Runnable.
   130  func (p *ddlJobPullerImpl) Run(ctx context.Context, _ ...chan<- error) error {
   131  	eg, ctx := errgroup.WithContext(ctx)
   132  
   133  	// Only nil in unit test.
   134  	if p.mp != nil {
   135  		eg.Go(func() error {
   136  			return p.mp.Run(ctx)
   137  		})
   138  	}
   139  
   140  	eg.Go(func() error {
   141  		return p.sorter.Run(ctx)
   142  	})
   143  
   144  	eg.Go(func() error {
   145  		for {
   146  			var sortedDDLEvent *model.PolymorphicEvent
   147  			select {
   148  			case <-ctx.Done():
   149  				return ctx.Err()
   150  			case sortedDDLEvent = <-p.sorter.Output():
   151  			}
   152  			if err := p.handleRawKVEntry(ctx, sortedDDLEvent.RawKV); err != nil {
   153  				return errors.Trace(err)
   154  			}
   155  		}
   156  	})
   157  
   158  	return eg.Wait()
   159  }
   160  
   161  // WaitForReady implements util.Runnable.
   162  func (p *ddlJobPullerImpl) WaitForReady(_ context.Context) {}
   163  
   164  // Close implements util.Runnable.
   165  func (p *ddlJobPullerImpl) Close() {}
   166  
   167  // Output implements DDLJobPuller, it returns the output channel of DDL job.
   168  func (p *ddlJobPullerImpl) Output() <-chan *model.DDLJobEntry {
   169  	return p.outputCh
   170  }
   171  
   172  // Input receives the raw kv entry and put it into the input channel.
   173  func (p *ddlJobPullerImpl) Input(
   174  	ctx context.Context,
   175  	rawDDL *model.RawKVEntry,
   176  	_ []tablepb.Span,
   177  ) error {
   178  	p.sorter.AddEntry(ctx, model.NewPolymorphicEvent(rawDDL))
   179  	return nil
   180  }
   181  
   182  // handleRawKVEntry converts the raw kv entry to DDL job and sends it to the output channel.
   183  func (p *ddlJobPullerImpl) handleRawKVEntry(ctx context.Context, ddlRawKV *model.RawKVEntry) error {
   184  	if ddlRawKV == nil {
   185  		return nil
   186  	}
   187  
   188  	if ddlRawKV.OpType == model.OpTypeResolved {
   189  		// Only nil in unit test case.
   190  		if p.schemaStorage != nil {
   191  			p.schemaStorage.AdvanceResolvedTs(ddlRawKV.CRTs)
   192  		}
   193  		if ddlRawKV.CRTs > p.getResolvedTs() {
   194  			p.setResolvedTs(ddlRawKV.CRTs)
   195  		}
   196  	}
   197  
   198  	job, err := p.unmarshalDDL(ddlRawKV)
   199  	if err != nil {
   200  		return errors.Trace(err)
   201  	}
   202  
   203  	if job != nil {
   204  		skip, err := p.handleJob(job)
   205  		if err != nil {
   206  			return err
   207  		}
   208  		if skip {
   209  			return nil
   210  		}
   211  		log.Info("a new ddl job is received",
   212  			zap.String("namespace", p.changefeedID.Namespace),
   213  			zap.String("changefeed", p.changefeedID.ID),
   214  			zap.String("schema", job.SchemaName),
   215  			zap.String("table", job.TableName),
   216  			zap.Uint64("startTs", job.StartTS),
   217  			zap.Uint64("finishedTs", job.BinlogInfo.FinishedTS),
   218  			zap.String("query", job.Query),
   219  			zap.Any("job", job))
   220  	}
   221  
   222  	jobEntry := &model.DDLJobEntry{
   223  		Job:    job,
   224  		OpType: ddlRawKV.OpType,
   225  		CRTs:   ddlRawKV.CRTs,
   226  	}
   227  	select {
   228  	case <-ctx.Done():
   229  		return ctx.Err()
   230  	case p.outputCh <- jobEntry:
   231  	}
   232  	return nil
   233  }
   234  
   235  func (p *ddlJobPullerImpl) unmarshalDDL(rawKV *model.RawKVEntry) (*timodel.Job, error) {
   236  	if rawKV.OpType != model.OpTypePut {
   237  		return nil, nil
   238  	}
   239  	if p.ddlTableInfo == nil && !entry.IsLegacyFormatJob(rawKV) {
   240  		err := p.initDDLTableInfo()
   241  		if err != nil {
   242  			return nil, errors.Trace(err)
   243  		}
   244  	}
   245  
   246  	return entry.ParseDDLJob(rawKV, p.ddlTableInfo)
   247  }
   248  
   249  func (p *ddlJobPullerImpl) getResolvedTs() uint64 {
   250  	return atomic.LoadUint64(&p.resolvedTs)
   251  }
   252  
   253  func (p *ddlJobPullerImpl) setResolvedTs(ts uint64) {
   254  	atomic.StoreUint64(&p.resolvedTs, ts)
   255  }
   256  
   257  func (p *ddlJobPullerImpl) initDDLTableInfo() error {
   258  	version, err := p.kvStorage.CurrentVersion(tidbkv.GlobalTxnScope)
   259  	if err != nil {
   260  		return errors.Trace(err)
   261  	}
   262  	snap := kv.GetSnapshotMeta(p.kvStorage, version.Ver)
   263  
   264  	dbInfos, err := snap.ListDatabases()
   265  	if err != nil {
   266  		return cerror.WrapError(cerror.ErrMetaListDatabases, err)
   267  	}
   268  
   269  	db, err := findDBByName(dbInfos, mysql.SystemDB)
   270  	if err != nil {
   271  		return errors.Trace(err)
   272  	}
   273  
   274  	tbls, err := snap.ListTables(db.ID)
   275  	if err != nil {
   276  		return errors.Trace(err)
   277  	}
   278  
   279  	// for tidb_ddl_job
   280  	tableInfo, err := findTableByName(tbls, "tidb_ddl_job")
   281  	if err != nil {
   282  		return errors.Trace(err)
   283  	}
   284  
   285  	col, err := findColumnByName(tableInfo.Columns, "job_meta")
   286  	if err != nil {
   287  		return errors.Trace(err)
   288  	}
   289  
   290  	p.ddlTableInfo = &entry.DDLTableInfo{}
   291  	p.ddlTableInfo.DDLJobTable = model.WrapTableInfo(db.ID, db.Name.L, 0, tableInfo)
   292  	p.ddlTableInfo.JobMetaColumnIDinJobTable = col.ID
   293  
   294  	// for tidb_ddl_history
   295  	historyTableInfo, err := findTableByName(tbls, "tidb_ddl_history")
   296  	if err != nil {
   297  		return errors.Trace(err)
   298  	}
   299  
   300  	historyTableCol, err := findColumnByName(historyTableInfo.Columns, "job_meta")
   301  	if err != nil {
   302  		return errors.Trace(err)
   303  	}
   304  
   305  	p.ddlTableInfo.DDLHistoryTable = model.WrapTableInfo(db.ID, db.Name.L, 0, historyTableInfo)
   306  	p.ddlTableInfo.JobMetaColumnIDinHistoryTable = historyTableCol.ID
   307  
   308  	return nil
   309  }
   310  
   311  // handleJob determines whether to filter out the DDL job.
   312  // If the DDL job is not filtered out, it will be applied to the schemaStorage
   313  // and the job will be sent to the output channel.
   314  func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) {
   315  	// Only nil in test.
   316  	if p.schemaStorage == nil {
   317  		return false, nil
   318  	}
   319  
   320  	if job.BinlogInfo.FinishedTS <= p.getResolvedTs() ||
   321  		job.BinlogInfo.SchemaVersion <= p.schemaVersion {
   322  		log.Info("ddl job finishedTs less than puller resolvedTs,"+
   323  			"discard the ddl job",
   324  			zap.String("namespace", p.changefeedID.Namespace),
   325  			zap.String("changefeed", p.changefeedID.ID),
   326  			zap.String("schema", job.SchemaName),
   327  			zap.String("table", job.TableName),
   328  			zap.Uint64("startTs", job.StartTS),
   329  			zap.Uint64("finishedTs", job.BinlogInfo.FinishedTS),
   330  			zap.String("query", job.Query),
   331  			zap.Uint64("pullerResolvedTs", p.getResolvedTs()))
   332  		return true, nil
   333  	}
   334  
   335  	defer func() {
   336  		if skip && err == nil {
   337  			log.Info("ddl job schema or table does not match, discard it",
   338  				zap.String("namespace", p.changefeedID.Namespace),
   339  				zap.String("changefeed", p.changefeedID.ID),
   340  				zap.String("schema", job.SchemaName),
   341  				zap.String("table", job.TableName),
   342  				zap.String("query", job.Query),
   343  				zap.Uint64("startTs", job.StartTS),
   344  				zap.Uint64("finishTs", job.BinlogInfo.FinishedTS))
   345  		}
   346  		if err != nil {
   347  			log.Warn("handle ddl job failed",
   348  				zap.String("namespace", p.changefeedID.Namespace),
   349  				zap.String("changefeed", p.changefeedID.ID),
   350  				zap.String("schema", job.SchemaName),
   351  				zap.String("table", job.TableName),
   352  				zap.String("query", job.Query),
   353  				zap.Uint64("startTs", job.StartTS),
   354  				zap.Uint64("finishTs", job.BinlogInfo.FinishedTS),
   355  				zap.Error(err))
   356  		}
   357  	}()
   358  
   359  	snap := p.schemaStorage.GetLastSnapshot()
   360  	if err = snap.FillSchemaName(job); err != nil {
   361  		log.Info("failed to fill schema name for ddl job",
   362  			zap.String("namespace", p.changefeedID.Namespace),
   363  			zap.String("changefeed", p.changefeedID.ID),
   364  			zap.String("schema", job.SchemaName),
   365  			zap.String("table", job.TableName),
   366  			zap.String("query", job.Query),
   367  			zap.Uint64("startTs", job.StartTS),
   368  			zap.Uint64("finishTs", job.BinlogInfo.FinishedTS),
   369  			zap.Error(err))
   370  		if p.filter.ShouldDiscardDDL(job.Type, job.SchemaName, job.TableName) {
   371  			return true, nil
   372  		}
   373  		return false, cerror.WrapError(cerror.ErrHandleDDLFailed,
   374  			errors.Trace(err), job.Query, job.StartTS, job.StartTS)
   375  	}
   376  
   377  	switch job.Type {
   378  	case timodel.ActionRenameTables:
   379  		skip, err = p.handleRenameTables(job)
   380  		if err != nil {
   381  			log.Warn("handle rename tables ddl job failed",
   382  				zap.String("namespace", p.changefeedID.Namespace),
   383  				zap.String("changefeed", p.changefeedID.ID),
   384  				zap.String("schema", job.SchemaName),
   385  				zap.String("table", job.TableName),
   386  				zap.String("query", job.Query),
   387  				zap.Uint64("startTs", job.StartTS),
   388  				zap.Uint64("finishTs", job.BinlogInfo.FinishedTS),
   389  				zap.Error(err))
   390  			return false, cerror.WrapError(cerror.ErrHandleDDLFailed,
   391  				errors.Trace(err), job.Query, job.StartTS, job.StartTS)
   392  		}
   393  	case timodel.ActionRenameTable:
   394  		oldTable, ok := snap.PhysicalTableByID(job.TableID)
   395  		if !ok {
   396  			// 1. If we can not find the old table, and the new table name is in filter rule, return error.
   397  			discard := p.filter.ShouldDiscardDDL(job.Type, job.SchemaName, job.BinlogInfo.TableInfo.Name.O)
   398  			if !discard {
   399  				return false, cerror.ErrSyncRenameTableFailed.GenWithStackByArgs(job.TableID, job.Query)
   400  			}
   401  			log.Warn("skip rename table ddl since cannot found the old table info",
   402  				zap.String("namespace", p.changefeedID.Namespace),
   403  				zap.String("changefeed", p.changefeedID.ID),
   404  				zap.Int64("tableID", job.TableID),
   405  				zap.Int64("newSchemaID", job.SchemaID),
   406  				zap.String("newSchemaName", job.SchemaName),
   407  				zap.String("oldTableName", job.BinlogInfo.TableInfo.Name.O),
   408  				zap.String("newTableName", job.TableName))
   409  			return true, nil
   410  		}
   411  		// since we can find the old table, it must be able to find the old schema.
   412  		// 2. If we can find the preTableInfo, we filter it by the old table name.
   413  		skipByOldTableName := p.filter.ShouldDiscardDDL(job.Type, oldTable.TableName.Schema, oldTable.TableName.Table)
   414  		skipByNewTableName := p.filter.ShouldDiscardDDL(job.Type, job.SchemaName, job.BinlogInfo.TableInfo.Name.O)
   415  		if err != nil {
   416  			return false, cerror.WrapError(cerror.ErrHandleDDLFailed,
   417  				errors.Trace(err), job.Query, job.StartTS, job.StartTS)
   418  		}
   419  		// 3. If its old table name is not in filter rule, and its new table name in filter rule, return error.
   420  		if skipByOldTableName {
   421  			if !skipByNewTableName {
   422  				return false, cerror.ErrSyncRenameTableFailed.GenWithStackByArgs(job.TableID, job.Query)
   423  			}
   424  			return true, nil
   425  		}
   426  		log.Info("ddl puller receive rename table ddl job",
   427  			zap.String("namespace", p.changefeedID.Namespace),
   428  			zap.String("changefeed", p.changefeedID.ID),
   429  			zap.String("schema", job.SchemaName),
   430  			zap.String("table", job.TableName),
   431  			zap.String("query", job.Query),
   432  			zap.Uint64("startTs", job.StartTS),
   433  			zap.Uint64("finishedTs", job.BinlogInfo.FinishedTS))
   434  	default:
   435  		// nil means it is a schema ddl job, it's no need to fill the table name.
   436  		if job.BinlogInfo.TableInfo != nil {
   437  			job.TableName = job.BinlogInfo.TableInfo.Name.O
   438  		}
   439  		skip = p.filter.ShouldDiscardDDL(job.Type, job.SchemaName, job.TableName)
   440  	}
   441  
   442  	if skip {
   443  		return true, nil
   444  	}
   445  
   446  	err = p.schemaStorage.HandleDDLJob(job)
   447  	if err != nil {
   448  		return false, cerror.WrapError(cerror.ErrHandleDDLFailed,
   449  			errors.Trace(err), job.Query, job.StartTS, job.StartTS)
   450  	}
   451  	p.setResolvedTs(job.BinlogInfo.FinishedTS)
   452  	p.schemaVersion = job.BinlogInfo.SchemaVersion
   453  
   454  	return p.checkIneligibleTableDDL(snap, job)
   455  }
   456  
   457  // checkIneligibleTableDDL checks if the table is ineligible before and after the DDL.
   458  //  1. If it is not a table DDL, we shouldn't check it.
   459  //  2. If the table after the DDL is ineligible:
   460  //     a. If the table is not exist before the DDL, we should ignore the DDL.
   461  //     b. If the table is ineligible before the DDL, we should ignore the DDL.
   462  //     c. If the table is eligible before the DDL, we should return an error.
   463  func (p *ddlJobPullerImpl) checkIneligibleTableDDL(snapBefore *schema.Snapshot, job *timodel.Job) (skip bool, err error) {
   464  	if filter.IsSchemaDDL(job.Type) {
   465  		return false, nil
   466  	}
   467  
   468  	ineligible := p.schemaStorage.GetLastSnapshot().IsIneligibleTableID(job.TableID)
   469  	if !ineligible {
   470  		return false, nil
   471  	}
   472  
   473  	// If the table is not in the snapshot before the DDL,
   474  	// we should ignore the DDL.
   475  	_, exist := snapBefore.PhysicalTableByID(job.TableID)
   476  	if !exist {
   477  		return true, nil
   478  	}
   479  
   480  	// If the table after the DDL is ineligible, we should check if it is not ineligible before the DDL.
   481  	// If so, we should return an error to inform the user that it is a
   482  	// dangerous operation and should be handled manually.
   483  	isBeforeineligible := snapBefore.IsIneligibleTableID(job.TableID)
   484  	if isBeforeineligible {
   485  		log.Warn("ignore the DDL event of ineligible table",
   486  			zap.String("changefeed", p.changefeedID.ID), zap.Any("ddl", job))
   487  		return true, nil
   488  	}
   489  	return false, cerror.New(fmt.Sprintf("An eligible table become ineligible after DDL: [%s] "+
   490  		"it is a dangerous operation and may cause data loss. If you want to replicate this ddl safely, "+
   491  		"pelase pause the changefeed and update the `force-replicate=true` "+
   492  		"in the changefeed configuration, "+
   493  		"then resume the changefeed.", job.Query))
   494  }
   495  
   496  // handleRenameTables gets all the tables that are renamed
   497  // in the DDL job out and filter them one by one,
   498  // if all the tables are filtered, skip it.
   499  func (p *ddlJobPullerImpl) handleRenameTables(job *timodel.Job) (skip bool, err error) {
   500  	var (
   501  		oldSchemaIDs, newSchemaIDs, oldTableIDs []int64
   502  		newTableNames, oldSchemaNames           []*timodel.CIStr
   503  	)
   504  
   505  	err = job.DecodeArgs(&oldSchemaIDs, &newSchemaIDs,
   506  		&newTableNames, &oldTableIDs, &oldSchemaNames)
   507  	if err != nil {
   508  		return true, errors.Trace(err)
   509  	}
   510  
   511  	var (
   512  		remainOldSchemaIDs, remainNewSchemaIDs, remainOldTableIDs []int64
   513  		remainNewTableNames, remainOldSchemaNames                 []*timodel.CIStr
   514  	)
   515  
   516  	multiTableInfos := job.BinlogInfo.MultipleTableInfos
   517  	if len(multiTableInfos) != len(oldSchemaIDs) ||
   518  		len(multiTableInfos) != len(newSchemaIDs) ||
   519  		len(multiTableInfos) != len(newTableNames) ||
   520  		len(multiTableInfos) != len(oldTableIDs) ||
   521  		len(multiTableInfos) != len(oldSchemaNames) {
   522  		return true, cerror.ErrInvalidDDLJob.GenWithStackByArgs(job.ID)
   523  	}
   524  
   525  	// we filter subordinate rename table ddl by these principles:
   526  	// 1. old table name matches the filter rule, remain it.
   527  	// 2. old table name does not match and new table name matches the filter rule, return error.
   528  	// 3. old table name and new table name do not match the filter rule, skip it.
   529  	remainTables := make([]*timodel.TableInfo, 0, len(multiTableInfos))
   530  	snap := p.schemaStorage.GetLastSnapshot()
   531  	for i, tableInfo := range multiTableInfos {
   532  		var shouldDiscardOldTable, shouldDiscardNewTable bool
   533  		oldTable, ok := snap.PhysicalTableByID(tableInfo.ID)
   534  		if !ok {
   535  			shouldDiscardOldTable = true
   536  		} else {
   537  			shouldDiscardOldTable = p.filter.ShouldDiscardDDL(job.Type, oldSchemaNames[i].O, oldTable.Name.O)
   538  		}
   539  
   540  		newSchemaName, ok := snap.SchemaByID(newSchemaIDs[i])
   541  		if !ok {
   542  			// the new table name does not hit the filter rule, so we should discard the table.
   543  			shouldDiscardNewTable = true
   544  		} else {
   545  			shouldDiscardNewTable = p.filter.ShouldDiscardDDL(job.Type, newSchemaName.Name.O, newTableNames[i].O)
   546  		}
   547  
   548  		if shouldDiscardOldTable && shouldDiscardNewTable {
   549  			// skip a rename table ddl only when its old table name and new table name are both filtered.
   550  			log.Info("RenameTables is filtered",
   551  				zap.Int64("tableID", tableInfo.ID),
   552  				zap.String("schema", oldSchemaNames[i].O),
   553  				zap.String("query", job.Query))
   554  			continue
   555  		}
   556  		if shouldDiscardOldTable && !shouldDiscardNewTable {
   557  			// if old table is not in filter rule and its new name is in filter rule, return error.
   558  			return true, cerror.ErrSyncRenameTableFailed.GenWithStackByArgs(tableInfo.ID, job.Query)
   559  		}
   560  		// old table name matches the filter rule, remain it.
   561  		remainTables = append(remainTables, tableInfo)
   562  		remainOldSchemaIDs = append(remainOldSchemaIDs, oldSchemaIDs[i])
   563  		remainNewSchemaIDs = append(remainNewSchemaIDs, newSchemaIDs[i])
   564  		remainOldTableIDs = append(remainOldTableIDs, oldTableIDs[i])
   565  		remainNewTableNames = append(remainNewTableNames, newTableNames[i])
   566  		remainOldSchemaNames = append(remainOldSchemaNames, oldSchemaNames[i])
   567  	}
   568  
   569  	if len(remainTables) == 0 {
   570  		return true, nil
   571  	}
   572  
   573  	newArgs := make([]json.RawMessage, 5)
   574  	v, err := json.Marshal(remainOldSchemaIDs)
   575  	if err != nil {
   576  		return true, errors.Trace(err)
   577  	}
   578  	newArgs[0] = v
   579  	v, err = json.Marshal(remainNewSchemaIDs)
   580  	if err != nil {
   581  		return true, errors.Trace(err)
   582  	}
   583  	newArgs[1] = v
   584  	v, err = json.Marshal(remainNewTableNames)
   585  	if err != nil {
   586  		return true, errors.Trace(err)
   587  	}
   588  	newArgs[2] = v
   589  	v, err = json.Marshal(remainOldTableIDs)
   590  	if err != nil {
   591  		return true, errors.Trace(err)
   592  	}
   593  	newArgs[3] = v
   594  	v, err = json.Marshal(remainOldSchemaNames)
   595  	if err != nil {
   596  		return true, errors.Trace(err)
   597  	}
   598  	newArgs[4] = v
   599  
   600  	newRawArgs, err := json.Marshal(newArgs)
   601  	if err != nil {
   602  		return true, errors.Trace(err)
   603  	}
   604  	job.RawArgs = newRawArgs
   605  	job.BinlogInfo.MultipleTableInfos = remainTables
   606  	return false, nil
   607  }
   608  
   609  // DDLPuller is the interface for DDL Puller, used by owner only.
   610  type DDLPuller interface {
   611  	// Run runs the DDLPuller
   612  	Run(ctx context.Context) error
   613  	// PopFrontDDL returns and pops the first DDL job in the internal queue
   614  	PopFrontDDL() (uint64, *timodel.Job)
   615  	// ResolvedTs returns the resolved ts of the DDLPuller
   616  	ResolvedTs() uint64
   617  	// Close closes the DDLPuller
   618  	Close()
   619  }
   620  
   621  type ddlPullerImpl struct {
   622  	ddlJobPuller DDLJobPuller
   623  
   624  	mu             sync.Mutex
   625  	resolvedTS     uint64
   626  	pendingDDLJobs []*timodel.Job
   627  	lastDDLJobID   int64
   628  	cancel         context.CancelFunc
   629  
   630  	changefeedID model.ChangeFeedID
   631  }
   632  
   633  // NewDDLPuller return a puller for DDL Event
   634  func NewDDLPuller(ctx context.Context,
   635  	up *upstream.Upstream,
   636  	startTs uint64,
   637  	changefeed model.ChangeFeedID,
   638  	schemaStorage entry.SchemaStorage,
   639  	filter filter.Filter,
   640  ) DDLPuller {
   641  	var puller DDLJobPuller
   642  	// storage can be nil only in the test
   643  	if up.KVStorage != nil {
   644  		changefeed.ID += "_owner_ddl_puller"
   645  		puller = NewDDLJobPuller(
   646  			ctx, up, startTs, config.GetGlobalServerConfig(),
   647  			changefeed, schemaStorage, filter)
   648  	}
   649  
   650  	return &ddlPullerImpl{
   651  		ddlJobPuller: puller,
   652  		resolvedTS:   startTs,
   653  		cancel:       func() {},
   654  		changefeedID: changefeed,
   655  	}
   656  }
   657  
   658  func (h *ddlPullerImpl) addToPending(job *timodel.Job) {
   659  	if job == nil {
   660  		return
   661  	}
   662  	if job.ID == h.lastDDLJobID {
   663  		log.Warn("ignore duplicated DDL job",
   664  			zap.String("namespace", h.changefeedID.Namespace),
   665  			zap.String("changefeed", h.changefeedID.ID),
   666  			zap.String("schema", job.SchemaName),
   667  			zap.String("table", job.TableName),
   668  
   669  			zap.String("query", job.Query),
   670  			zap.Uint64("startTs", job.StartTS),
   671  			zap.Uint64("finishTs", job.BinlogInfo.FinishedTS),
   672  			zap.Int64("jobID", job.ID))
   673  		return
   674  	}
   675  	h.mu.Lock()
   676  	defer h.mu.Unlock()
   677  	h.pendingDDLJobs = append(h.pendingDDLJobs, job)
   678  	h.lastDDLJobID = job.ID
   679  	log.Info("ddl puller receives new pending job",
   680  		zap.String("namespace", h.changefeedID.Namespace),
   681  		zap.String("changefeed", h.changefeedID.ID),
   682  		zap.String("schema", job.SchemaName),
   683  		zap.String("table", job.TableName),
   684  		zap.String("query", job.Query),
   685  		zap.Uint64("startTs", job.StartTS),
   686  		zap.Uint64("finishTs", job.BinlogInfo.FinishedTS),
   687  		zap.Int64("jobID", job.ID))
   688  }
   689  
   690  // Run the ddl puller to receive DDL events
   691  func (h *ddlPullerImpl) Run(ctx context.Context) error {
   692  	g, ctx := errgroup.WithContext(ctx)
   693  	ctx, cancel := context.WithCancel(ctx)
   694  	h.cancel = cancel
   695  
   696  	g.Go(func() error { return h.ddlJobPuller.Run(ctx) })
   697  
   698  	g.Go(func() error {
   699  		cc := clock.New()
   700  		ticker := cc.Ticker(ddlPullerStuckWarnDuration)
   701  		defer ticker.Stop()
   702  		lastResolvedTsAdvancedTime := cc.Now()
   703  		for {
   704  			select {
   705  			case <-ctx.Done():
   706  				return ctx.Err()
   707  			case <-ticker.C:
   708  				duration := cc.Since(lastResolvedTsAdvancedTime)
   709  				if duration > ddlPullerStuckWarnDuration {
   710  					log.Warn("ddl puller resolved ts has not advanced",
   711  						zap.String("namespace", h.changefeedID.Namespace),
   712  						zap.String("changefeed", h.changefeedID.ID),
   713  						zap.Duration("duration", duration),
   714  						zap.Uint64("resolvedTs", atomic.LoadUint64(&h.resolvedTS)))
   715  				}
   716  			case e := <-h.ddlJobPuller.Output():
   717  				if e.OpType == model.OpTypeResolved {
   718  					if e.CRTs > atomic.LoadUint64(&h.resolvedTS) {
   719  						atomic.StoreUint64(&h.resolvedTS, e.CRTs)
   720  						lastResolvedTsAdvancedTime = cc.Now()
   721  						continue
   722  					}
   723  				}
   724  				h.addToPending(e.Job)
   725  			}
   726  		}
   727  	})
   728  
   729  	log.Info("DDL puller started",
   730  		zap.String("namespace", h.changefeedID.Namespace),
   731  		zap.String("changefeed", h.changefeedID.ID),
   732  		zap.Uint64("resolvedTS", atomic.LoadUint64(&h.resolvedTS)))
   733  
   734  	return g.Wait()
   735  }
   736  
   737  // PopFrontDDL return the first pending DDL job and remove it from the pending list
   738  func (h *ddlPullerImpl) PopFrontDDL() (uint64, *timodel.Job) {
   739  	h.mu.Lock()
   740  	defer h.mu.Unlock()
   741  	if len(h.pendingDDLJobs) == 0 {
   742  		return atomic.LoadUint64(&h.resolvedTS), nil
   743  	}
   744  	job := h.pendingDDLJobs[0]
   745  	h.pendingDDLJobs = h.pendingDDLJobs[1:]
   746  	return job.BinlogInfo.FinishedTS, job
   747  }
   748  
   749  // Close the ddl puller, release all resources.
   750  func (h *ddlPullerImpl) Close() {
   751  	log.Info("close the ddl puller",
   752  		zap.String("namespace", h.changefeedID.Namespace),
   753  		zap.String("changefeed", h.changefeedID.ID))
   754  	h.cancel()
   755  }
   756  
   757  func (h *ddlPullerImpl) ResolvedTs() uint64 {
   758  	h.mu.Lock()
   759  	defer h.mu.Unlock()
   760  	if len(h.pendingDDLJobs) == 0 {
   761  		return atomic.LoadUint64(&h.resolvedTS)
   762  	}
   763  	job := h.pendingDDLJobs[0]
   764  	return job.BinlogInfo.FinishedTS
   765  }
   766  
   767  // Below are some helper functions for ddl puller.
   768  func findDBByName(dbs []*timodel.DBInfo, name string) (*timodel.DBInfo, error) {
   769  	for _, db := range dbs {
   770  		if db.Name.L == name {
   771  			return db, nil
   772  		}
   773  	}
   774  	return nil, cerror.WrapError(
   775  		cerror.ErrDDLSchemaNotFound,
   776  		errors.Errorf("can't find schema %s", name))
   777  }
   778  
   779  func findTableByName(tbls []*timodel.TableInfo, name string) (*timodel.TableInfo, error) {
   780  	for _, t := range tbls {
   781  		if t.Name.L == name {
   782  			return t, nil
   783  		}
   784  	}
   785  	return nil, cerror.WrapError(
   786  		cerror.ErrDDLSchemaNotFound,
   787  		errors.Errorf("can't find table %s", name))
   788  }
   789  
   790  func findColumnByName(cols []*timodel.ColumnInfo, name string) (*timodel.ColumnInfo, error) {
   791  	for _, c := range cols {
   792  		if c.Name.L == name {
   793  			return c, nil
   794  		}
   795  	}
   796  	return nil, cerror.WrapError(
   797  		cerror.ErrDDLSchemaNotFound,
   798  		errors.Errorf("can't find column %s", name))
   799  }