github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/owner/ddl_sink.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 owner
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"strings"
    20  	"sync"
    21  	"time"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/failpoint"
    25  	"github.com/pingcap/log"
    26  	"github.com/pingcap/tidb/pkg/parser"
    27  	"github.com/pingcap/tidb/pkg/parser/format"
    28  	"github.com/pingcap/tiflow/cdc/model"
    29  	"github.com/pingcap/tiflow/cdc/sink/ddlsink"
    30  	"github.com/pingcap/tiflow/cdc/sink/ddlsink/factory"
    31  	"github.com/pingcap/tiflow/cdc/syncpointstore"
    32  	cerror "github.com/pingcap/tiflow/pkg/errors"
    33  	"github.com/pingcap/tiflow/pkg/retry"
    34  	"github.com/pingcap/tiflow/pkg/util"
    35  	"go.uber.org/zap"
    36  )
    37  
    38  const (
    39  	defaultErrChSize = 1024
    40  )
    41  
    42  // DDLSink is a wrapper of the `Sink` interface for the owner
    43  // DDLSink should send `DDLEvent` and `CheckpointTs` to downstream,
    44  // If `SyncPointEnabled`, also send `syncPoint` to downstream.
    45  type DDLSink interface {
    46  	// run the DDLSink
    47  	run(ctx context.Context)
    48  	// emitCheckpointTs emits the checkpoint Ts to downstream data source
    49  	// this function will return after recording the checkpointTs specified in memory immediately
    50  	// and the recorded checkpointTs will be sent and updated to downstream data source every second
    51  	emitCheckpointTs(ts uint64, tables []*model.TableInfo)
    52  	// emitDDLEvent emits DDL event and return true if the DDL is executed
    53  	// the DDL event will be sent to another goroutine and execute to downstream
    54  	// the caller of this function can call again and again until a true returned
    55  	emitDDLEvent(ctx context.Context, ddl *model.DDLEvent) (bool, error)
    56  	emitSyncPoint(ctx context.Context, checkpointTs uint64) error
    57  	// close the ddlsink, cancel running goroutine.
    58  	close(ctx context.Context) error
    59  }
    60  
    61  type ddlSinkImpl struct {
    62  	lastSyncPoint  model.Ts
    63  	syncPointStore syncpointstore.SyncPointStore
    64  
    65  	// It is used to record the checkpointTs and the names of the table at that time.
    66  	mu struct {
    67  		sync.Mutex
    68  		checkpointTs  model.Ts
    69  		currentTables []*model.TableInfo
    70  	}
    71  	// ddlSentTsMap is used to check whether a ddl event in a ddl job has been
    72  	// sent to `ddlCh` successfully.
    73  	ddlSentTsMap map[*model.DDLEvent]model.Ts
    74  
    75  	ddlCh chan *model.DDLEvent
    76  
    77  	sink ddlsink.Sink
    78  	// `sinkInitHandler` can be helpful in unit testing.
    79  	sinkInitHandler ddlSinkInitHandler
    80  
    81  	// cancel would be used to cancel the goroutine start by `run`
    82  	cancel context.CancelFunc
    83  	wg     sync.WaitGroup
    84  
    85  	changefeedID model.ChangeFeedID
    86  	info         *model.ChangeFeedInfo
    87  
    88  	sinkRetry     *retry.ErrorRetry
    89  	reportError   func(err error)
    90  	reportWarning func(err error)
    91  }
    92  
    93  func newDDLSink(
    94  	changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo,
    95  	reportError func(err error), reportWarning func(err error),
    96  ) DDLSink {
    97  	res := &ddlSinkImpl{
    98  		ddlSentTsMap:    make(map[*model.DDLEvent]uint64),
    99  		ddlCh:           make(chan *model.DDLEvent, 1),
   100  		sinkInitHandler: ddlSinkInitializer,
   101  		cancel:          func() {},
   102  
   103  		changefeedID: changefeedID,
   104  		info:         info,
   105  
   106  		sinkRetry:     retry.NewInfiniteErrorRetry(),
   107  		reportError:   reportError,
   108  		reportWarning: reportWarning,
   109  	}
   110  	return res
   111  }
   112  
   113  type ddlSinkInitHandler func(ctx context.Context, a *ddlSinkImpl) error
   114  
   115  func ddlSinkInitializer(ctx context.Context, a *ddlSinkImpl) error {
   116  	log.Info("Try to create ddlSink based on sink",
   117  		zap.String("namespace", a.changefeedID.Namespace),
   118  		zap.String("changefeed", a.changefeedID.ID))
   119  	s, err := factory.New(ctx, a.changefeedID, a.info.SinkURI, a.info.Config)
   120  	if err != nil {
   121  		return errors.Trace(err)
   122  	}
   123  	a.sink = s
   124  
   125  	if !util.GetOrZero(a.info.Config.EnableSyncPoint) {
   126  		return nil
   127  	}
   128  	return nil
   129  }
   130  
   131  func (s *ddlSinkImpl) makeSyncPointStoreReady(ctx context.Context) error {
   132  	if util.GetOrZero(s.info.Config.EnableSyncPoint) && s.syncPointStore == nil {
   133  		syncPointStore, err := syncpointstore.NewSyncPointStore(
   134  			ctx, s.changefeedID, s.info.SinkURI, util.GetOrZero(s.info.Config.SyncPointRetention))
   135  		if err != nil {
   136  			return errors.Trace(err)
   137  		}
   138  		failpoint.Inject("DDLSinkInitializeSlowly", func() {
   139  			time.Sleep(time.Second * 5)
   140  		})
   141  		s.syncPointStore = syncPointStore
   142  
   143  		if err := s.syncPointStore.CreateSyncTable(ctx); err != nil {
   144  			return errors.Trace(err)
   145  		}
   146  	}
   147  	return nil
   148  }
   149  
   150  func (s *ddlSinkImpl) makeSinkReady(ctx context.Context) error {
   151  	if s.sink == nil {
   152  		if err := s.sinkInitHandler(ctx, s); err != nil {
   153  			log.Warn("ddl sink initialize failed",
   154  				zap.String("namespace", s.changefeedID.Namespace),
   155  				zap.String("changefeed", s.changefeedID.ID),
   156  				zap.Error(err))
   157  			return errors.New("ddlSink not ready")
   158  		}
   159  	}
   160  	return nil
   161  }
   162  
   163  // retry the given action with 5s interval. Before every retry, s.sink will be re-initialized.
   164  func (s *ddlSinkImpl) retrySinkAction(ctx context.Context, name string, action func() error) (err error) {
   165  	for {
   166  		if err = action(); err == nil {
   167  			return nil
   168  		}
   169  		isRetryable := !cerror.ShouldFailChangefeed(err) && errors.Cause(err) != context.Canceled
   170  		log.Warn("owner ddl sink fails on action",
   171  			zap.String("namespace", s.changefeedID.Namespace),
   172  			zap.String("changefeed", s.changefeedID.ID),
   173  			zap.String("action", name),
   174  			zap.Bool("retryable", isRetryable),
   175  			zap.Error(err))
   176  
   177  		s.sink = nil
   178  		if isRetryable {
   179  			s.reportWarning(err)
   180  		} else {
   181  			s.reportError(err)
   182  			return err
   183  		}
   184  
   185  		backoff, err := s.sinkRetry.GetRetryBackoff(err)
   186  		if err != nil {
   187  			return errors.New(fmt.Sprintf("GetRetryBackoff: %s", err.Error()))
   188  		}
   189  
   190  		if err = util.Hang(ctx, backoff); err != nil {
   191  			return errors.Trace(err)
   192  		}
   193  	}
   194  }
   195  
   196  func (s *ddlSinkImpl) observedRetrySinkAction(ctx context.Context, name string, action func() error) (err error) {
   197  	errCh := make(chan error, 1)
   198  	go func() { errCh <- s.retrySinkAction(ctx, name, action) }()
   199  	ticker := time.NewTicker(30 * time.Second)
   200  	defer ticker.Stop()
   201  	for {
   202  		select {
   203  		case err := <-errCh:
   204  			return err
   205  		case <-ticker.C:
   206  			log.Info("owner ddl sink performs an action too long",
   207  				zap.String("namespace", s.changefeedID.Namespace),
   208  				zap.String("changefeed", s.changefeedID.ID),
   209  				zap.String("action", name))
   210  		}
   211  	}
   212  }
   213  
   214  func (s *ddlSinkImpl) writeCheckpointTs(ctx context.Context, lastCheckpointTs *model.Ts) error {
   215  	doWrite := func() (err error) {
   216  		s.mu.Lock()
   217  		checkpointTs := s.mu.checkpointTs
   218  		if checkpointTs == 0 || checkpointTs <= *lastCheckpointTs {
   219  			s.mu.Unlock()
   220  			return
   221  		}
   222  		tables := make([]*model.TableInfo, 0, len(s.mu.currentTables))
   223  		tables = append(tables, s.mu.currentTables...)
   224  		s.mu.Unlock()
   225  
   226  		if err = s.makeSinkReady(ctx); err == nil {
   227  			err = s.sink.WriteCheckpointTs(ctx, checkpointTs, tables)
   228  		}
   229  		if err == nil {
   230  			*lastCheckpointTs = checkpointTs
   231  		}
   232  		return
   233  	}
   234  
   235  	return s.observedRetrySinkAction(ctx, "writeCheckpointTs", doWrite)
   236  }
   237  
   238  func (s *ddlSinkImpl) writeDDLEvent(ctx context.Context, ddl *model.DDLEvent) error {
   239  	log.Info("begin emit ddl event",
   240  		zap.String("namespace", s.changefeedID.Namespace),
   241  		zap.String("changefeed", s.changefeedID.ID),
   242  		zap.Any("DDL", ddl))
   243  
   244  	doWrite := func() (err error) {
   245  		if err = s.makeSinkReady(ctx); err == nil {
   246  			err = s.sink.WriteDDLEvent(ctx, ddl)
   247  			failpoint.Inject("InjectChangefeedDDLError", func() {
   248  				err = cerror.ErrExecDDLFailed.GenWithStackByArgs()
   249  			})
   250  		}
   251  		if err != nil {
   252  			log.Error("Execute DDL failed",
   253  				zap.String("namespace", s.changefeedID.Namespace),
   254  				zap.String("changefeed", s.changefeedID.ID),
   255  				zap.Any("DDL", ddl),
   256  				zap.Error(err))
   257  		} else {
   258  			ddl.Done.Store(true)
   259  			log.Info("Execute DDL succeeded",
   260  				zap.String("namespace", s.changefeedID.Namespace),
   261  				zap.String("changefeed", s.changefeedID.ID),
   262  				zap.Any("DDL", ddl))
   263  		}
   264  		return
   265  	}
   266  
   267  	return s.observedRetrySinkAction(ctx, "writeDDLEvent", doWrite)
   268  }
   269  
   270  func (s *ddlSinkImpl) run(ctx context.Context) {
   271  	ctx, s.cancel = context.WithCancel(ctx)
   272  
   273  	s.wg.Add(1)
   274  	go func() {
   275  		var err error
   276  		log.Info("owner ddl sink background loop is started",
   277  			zap.String("namespace", s.changefeedID.Namespace),
   278  			zap.String("changefeed", s.changefeedID.ID))
   279  		defer func() {
   280  			s.wg.Done()
   281  			log.Info("owner ddl sink background loop exits",
   282  				zap.String("namespace", s.changefeedID.Namespace),
   283  				zap.String("changefeed", s.changefeedID.ID),
   284  				zap.Error(err))
   285  		}()
   286  
   287  		// TODO make the tick duration configurable
   288  		ticker := time.NewTicker(time.Second)
   289  		defer ticker.Stop()
   290  		var lastCheckpointTs model.Ts
   291  		for {
   292  			// `ticker.C` and `ddlCh` may can be triggered at the same time, it
   293  			// does not matter which one emit first, since TiCDC allow DDL with
   294  			// CommitTs equal to the last CheckpointTs be emitted later.
   295  			select {
   296  			case <-ctx.Done():
   297  				err = ctx.Err()
   298  				return
   299  			case <-ticker.C:
   300  				if err = s.writeCheckpointTs(ctx, &lastCheckpointTs); err != nil {
   301  					return
   302  				}
   303  			case ddl := <-s.ddlCh:
   304  				if err = s.writeDDLEvent(ctx, ddl); err != nil {
   305  					return
   306  				}
   307  				// Force emitting checkpoint ts when a ddl event is finished.
   308  				// Otherwise, a kafka consumer may not execute that ddl event.
   309  				if err = s.writeCheckpointTs(ctx, &lastCheckpointTs); err != nil {
   310  					return
   311  				}
   312  			}
   313  		}
   314  	}()
   315  }
   316  
   317  func (s *ddlSinkImpl) emitCheckpointTs(ts uint64, tables []*model.TableInfo) {
   318  	s.mu.Lock()
   319  	defer s.mu.Unlock()
   320  	s.mu.checkpointTs = ts
   321  	s.mu.currentTables = tables
   322  }
   323  
   324  // emitDDLEvent returns true if the ddl event is already executed.
   325  // For the `rename tables` job, the events in that job have identical StartTs
   326  // and CommitTs. So in emitDDLEvent, we get the DDL finished ts of an event
   327  // from a map in order to check whether that event is finished or not.
   328  func (s *ddlSinkImpl) emitDDLEvent(ctx context.Context, ddl *model.DDLEvent) (bool, error) {
   329  	s.mu.Lock()
   330  	if ddl.Done.Load() {
   331  		// the DDL event is executed successfully, and done is true
   332  		log.Info("ddl already executed, skip it",
   333  			zap.String("namespace", s.changefeedID.Namespace),
   334  			zap.String("changefeed", s.changefeedID.ID),
   335  			zap.Any("DDL", ddl))
   336  		delete(s.ddlSentTsMap, ddl)
   337  		s.mu.Unlock()
   338  		return true, nil
   339  	}
   340  
   341  	ddlSentTs := s.ddlSentTsMap[ddl]
   342  	if ddl.CommitTs <= ddlSentTs {
   343  		log.Debug("ddl is not finished yet",
   344  			zap.String("namespace", s.changefeedID.Namespace),
   345  			zap.String("changefeed", s.changefeedID.ID),
   346  			zap.Uint64("ddlSentTs", ddlSentTs), zap.Any("DDL", ddl))
   347  		// the DDL event is executing and not finished yet, return false
   348  		s.mu.Unlock()
   349  		return false, nil
   350  	}
   351  
   352  	query, err := s.addSpecialComment(ddl)
   353  	if err != nil {
   354  		log.Error("Add special comment failed",
   355  			zap.String("namespace", s.changefeedID.Namespace),
   356  			zap.String("changefeed", s.changefeedID.ID),
   357  			zap.Error(err),
   358  			zap.Any("ddl", ddl))
   359  		s.mu.Unlock()
   360  		return false, errors.Trace(err)
   361  	}
   362  	ddl.Query = query
   363  	s.mu.Unlock()
   364  
   365  	select {
   366  	case <-ctx.Done():
   367  		return false, errors.Trace(ctx.Err())
   368  	case s.ddlCh <- ddl:
   369  		s.ddlSentTsMap[ddl] = ddl.CommitTs
   370  		log.Info("ddl is sent",
   371  			zap.String("namespace", s.changefeedID.Namespace),
   372  			zap.String("changefeed", s.changefeedID.ID),
   373  			zap.Uint64("ddlSentTs", ddl.CommitTs))
   374  	default:
   375  		log.Warn("ddl chan full, send it the next round",
   376  			zap.String("namespace", s.changefeedID.Namespace),
   377  			zap.String("changefeed", s.changefeedID.ID),
   378  			zap.Uint64("ddlSentTs", ddlSentTs),
   379  			zap.Any("DDL", ddl))
   380  		// if this hit, we think that ddlCh is full,
   381  		// just return false and send the ddl in the next round.
   382  	}
   383  	return false, nil
   384  }
   385  
   386  func (s *ddlSinkImpl) emitSyncPoint(ctx context.Context, checkpointTs uint64) (err error) {
   387  	if checkpointTs == s.lastSyncPoint {
   388  		return nil
   389  	}
   390  	s.lastSyncPoint = checkpointTs
   391  
   392  	for {
   393  		if err = s.makeSyncPointStoreReady(ctx); err == nil {
   394  			// TODO implement async sink syncPoint
   395  			err = s.syncPointStore.SinkSyncPoint(ctx, s.changefeedID, checkpointTs)
   396  		}
   397  		if err == nil {
   398  			return nil
   399  		}
   400  		if !cerror.ShouldFailChangefeed(err) && errors.Cause(err) != context.Canceled {
   401  			// TODO(qupeng): retry it internally after async sink syncPoint is ready.
   402  			s.reportError(err)
   403  			return err
   404  		}
   405  		s.reportError(err)
   406  		return err
   407  	}
   408  }
   409  
   410  func (s *ddlSinkImpl) close(ctx context.Context) (err error) {
   411  	s.cancel()
   412  	s.wg.Wait()
   413  
   414  	// they will both be nil if changefeed return an error in initializing
   415  	if s.sink != nil {
   416  		s.sink.Close()
   417  	}
   418  	if s.syncPointStore != nil {
   419  		err = s.syncPointStore.Close()
   420  	}
   421  	if err != nil && errors.Cause(err) != context.Canceled {
   422  		return err
   423  	}
   424  	return nil
   425  }
   426  
   427  // addSpecialComment translate tidb feature to comment
   428  func (s *ddlSinkImpl) addSpecialComment(ddl *model.DDLEvent) (string, error) {
   429  	p := parser.New()
   430  	// We need to use the correct SQL mode to parse the DDL query.
   431  	// Otherwise, the parser may fail to parse the DDL query.
   432  	// For example, it is needed to parse the following DDL query:
   433  	//  `alter table "t" add column "c" int default 1;`
   434  	// by adding `ANSI_QUOTES` to the SQL mode.
   435  	p.SetSQLMode(ddl.SQLMode)
   436  	stms, _, err := p.Parse(ddl.Query, ddl.Charset, ddl.Collate)
   437  	if err != nil {
   438  		return "", errors.Trace(err)
   439  	}
   440  	if len(stms) != 1 {
   441  		log.Error("invalid ddlQuery statement size",
   442  			zap.String("namespace", s.changefeedID.Namespace),
   443  			zap.String("changefeed", s.changefeedID.ID),
   444  			zap.String("ddlQuery", ddl.Query))
   445  		return "", cerror.ErrUnexpected.FastGenByArgs("invalid ddlQuery statement size")
   446  	}
   447  	var sb strings.Builder
   448  	// translate TiDB feature to special comment
   449  	restoreFlags := format.RestoreTiDBSpecialComment
   450  	// escape the keyword
   451  	restoreFlags |= format.RestoreNameBackQuotes
   452  	// upper case keyword
   453  	restoreFlags |= format.RestoreKeyWordUppercase
   454  	// wrap string with single quote
   455  	restoreFlags |= format.RestoreStringSingleQuotes
   456  	// remove placement rule
   457  	restoreFlags |= format.SkipPlacementRuleForRestore
   458  	// force disable ttl
   459  	restoreFlags |= format.RestoreWithTTLEnableOff
   460  	if err = stms[0].Restore(format.NewRestoreCtx(restoreFlags, &sb)); err != nil {
   461  		return "", errors.Trace(err)
   462  	}
   463  
   464  	result := sb.String()
   465  	log.Info("add special comment to DDL",
   466  		zap.String("namespace", s.changefeedID.Namespace),
   467  		zap.String("changefeed", s.changefeedID.ID),
   468  		zap.String("DDL", ddl.Query),
   469  		zap.String("charset", ddl.Charset),
   470  		zap.String("collate", ddl.Collate),
   471  		zap.String("result", result))
   472  
   473  	return result, nil
   474  }