github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/owner/changefeed.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  	"sync"
    19  	"time"
    20  
    21  	"github.com/pingcap/errors"
    22  	"github.com/pingcap/failpoint"
    23  	"github.com/pingcap/log"
    24  	timodel "github.com/pingcap/parser/model"
    25  	"github.com/pingcap/ticdc/cdc/model"
    26  	cdcContext "github.com/pingcap/ticdc/pkg/context"
    27  	cerror "github.com/pingcap/ticdc/pkg/errors"
    28  	"github.com/pingcap/ticdc/pkg/util"
    29  	"github.com/pingcap/tidb/sessionctx/binloginfo"
    30  	"github.com/pingcap/tidb/store/tikv/oracle"
    31  	"github.com/prometheus/client_golang/prometheus"
    32  	"go.uber.org/zap"
    33  )
    34  
    35  type changefeed struct {
    36  	id    model.ChangeFeedID
    37  	state *model.ChangefeedReactorState
    38  
    39  	scheduler        *scheduler
    40  	barriers         *barriers
    41  	feedStateManager *feedStateManager
    42  	gcManager        GcManager
    43  
    44  	schema      *schemaWrap4Owner
    45  	sink        AsyncSink
    46  	ddlPuller   DDLPuller
    47  	initialized bool
    48  
    49  	// only used for asyncExecDDL function
    50  	// ddlEventCache is not nil when the changefeed is executing a DDL event asynchronously
    51  	// After the DDL event has been executed, ddlEventCache will be set to nil.
    52  	ddlEventCache *model.DDLEvent
    53  
    54  	errCh  chan error
    55  	cancel context.CancelFunc
    56  
    57  	// The changefeed will start some backend goroutines in the function `initialize`,
    58  	// such as DDLPuller, Sink, etc.
    59  	// `wg` is used to manage those backend goroutines.
    60  	// But it only manages the DDLPuller for now.
    61  	// TODO: manage the Sink and other backend goroutines.
    62  	wg sync.WaitGroup
    63  
    64  	metricsChangefeedCheckpointTsGauge    prometheus.Gauge
    65  	metricsChangefeedCheckpointTsLagGauge prometheus.Gauge
    66  
    67  	newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error)
    68  	newSink      func(ctx cdcContext.Context) (AsyncSink, error)
    69  }
    70  
    71  func newChangefeed(id model.ChangeFeedID, gcManager GcManager) *changefeed {
    72  	c := &changefeed{
    73  		id:               id,
    74  		scheduler:        newScheduler(),
    75  		barriers:         newBarriers(),
    76  		feedStateManager: new(feedStateManager),
    77  		gcManager:        gcManager,
    78  
    79  		errCh:  make(chan error, defaultErrChSize),
    80  		cancel: func() {},
    81  
    82  		newDDLPuller: newDDLPuller,
    83  	}
    84  	c.newSink = newAsyncSink
    85  	return c
    86  }
    87  
    88  func newChangefeed4Test(
    89  	id model.ChangeFeedID, gcManager GcManager,
    90  	newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error),
    91  	newSink func(ctx cdcContext.Context) (AsyncSink, error),
    92  ) *changefeed {
    93  	c := newChangefeed(id, gcManager)
    94  	c.newDDLPuller = newDDLPuller
    95  	c.newSink = newSink
    96  	return c
    97  }
    98  
    99  func (c *changefeed) Tick(ctx cdcContext.Context, state *model.ChangefeedReactorState, captures map[model.CaptureID]*model.CaptureInfo) {
   100  	ctx = cdcContext.WithErrorHandler(ctx, func(err error) error {
   101  		c.errCh <- errors.Trace(err)
   102  		return nil
   103  	})
   104  	state.CheckCaptureAlive(ctx.GlobalVars().CaptureInfo.ID)
   105  	if err := c.tick(ctx, state, captures); err != nil {
   106  		log.Error("an error occurred in Owner", zap.String("changefeedID", c.state.ID), zap.Error(err))
   107  		var code string
   108  		if rfcCode, ok := cerror.RFCCode(err); ok {
   109  			code = string(rfcCode)
   110  		} else {
   111  			code = string(cerror.ErrOwnerUnknown.RFCCode())
   112  		}
   113  		c.feedStateManager.HandleError(&model.RunningError{
   114  			Addr:    util.CaptureAddrFromCtx(ctx),
   115  			Code:    code,
   116  			Message: err.Error(),
   117  		})
   118  		c.releaseResources()
   119  	}
   120  }
   121  
   122  func (c *changefeed) checkStaleCheckpointTs(ctx cdcContext.Context, checkpointTs uint64) error {
   123  	state := c.state.Info.State
   124  	if state == model.StateNormal || state == model.StateStopped || state == model.StateError {
   125  		if err := c.gcManager.checkStaleCheckpointTs(ctx, checkpointTs); err != nil {
   126  			return errors.Trace(err)
   127  		}
   128  	}
   129  	return nil
   130  }
   131  
   132  func (c *changefeed) tick(ctx cdcContext.Context, state *model.ChangefeedReactorState, captures map[model.CaptureID]*model.CaptureInfo) error {
   133  	c.state = state
   134  	c.feedStateManager.Tick(state)
   135  	if !c.feedStateManager.ShouldRunning() {
   136  		c.releaseResources()
   137  		return nil
   138  	}
   139  
   140  	checkpointTs := c.state.Info.GetCheckpointTs(c.state.Status)
   141  	if err := c.checkStaleCheckpointTs(ctx, checkpointTs); err != nil {
   142  		return errors.Trace(err)
   143  	}
   144  
   145  	if !c.preflightCheck(captures) {
   146  		return nil
   147  	}
   148  	if err := c.initialize(ctx); err != nil {
   149  		return errors.Trace(err)
   150  	}
   151  
   152  	select {
   153  	case err := <-c.errCh:
   154  		return errors.Trace(err)
   155  	default:
   156  	}
   157  
   158  	c.sink.EmitCheckpointTs(ctx, checkpointTs)
   159  	barrierTs, err := c.handleBarrier(ctx)
   160  	if err != nil {
   161  		return errors.Trace(err)
   162  	}
   163  	shouldUpdateState, err := c.scheduler.Tick(c.state, c.schema.AllPhysicalTables(), captures)
   164  	if err != nil {
   165  		return errors.Trace(err)
   166  	}
   167  	if shouldUpdateState {
   168  		c.updateStatus(barrierTs)
   169  	}
   170  	return nil
   171  }
   172  
   173  func (c *changefeed) initialize(ctx cdcContext.Context) error {
   174  	if c.initialized {
   175  		return nil
   176  	}
   177  	// clean the errCh
   178  	// When the changefeed is resumed after being stopped, the changefeed instance will be reused,
   179  	// So we should make sure that the errCh is empty when the changefeed is restarting
   180  LOOP:
   181  	for {
   182  		select {
   183  		case <-c.errCh:
   184  		default:
   185  			break LOOP
   186  		}
   187  	}
   188  	checkpointTs := c.state.Info.GetCheckpointTs(c.state.Status)
   189  	log.Info("initialize changefeed", zap.String("changefeed", c.state.ID),
   190  		zap.Stringer("info", c.state.Info),
   191  		zap.Uint64("checkpoint ts", checkpointTs))
   192  	failpoint.Inject("NewChangefeedNoRetryError", func() {
   193  		failpoint.Return(cerror.ErrStartTsBeforeGC.GenWithStackByArgs(checkpointTs-300, checkpointTs))
   194  	})
   195  	failpoint.Inject("NewChangefeedRetryError", func() {
   196  		failpoint.Return(errors.New("failpoint injected retriable error"))
   197  	})
   198  	if c.state.Info.Config.CheckGCSafePoint {
   199  		err := util.CheckSafetyOfStartTs(ctx, ctx.GlobalVars().PDClient, c.state.ID, checkpointTs)
   200  		if err != nil {
   201  			return errors.Trace(err)
   202  		}
   203  	}
   204  	if c.state.Info.SyncPointEnabled {
   205  		c.barriers.Update(syncPointBarrier, checkpointTs)
   206  	}
   207  	c.barriers.Update(ddlJobBarrier, checkpointTs)
   208  	c.barriers.Update(finishBarrier, c.state.Info.GetTargetTs())
   209  	var err error
   210  	c.schema, err = newSchemaWrap4Owner(ctx.GlobalVars().KVStorage, checkpointTs, c.state.Info.Config)
   211  	if err != nil {
   212  		return errors.Trace(err)
   213  	}
   214  	cancelCtx, cancel := cdcContext.WithCancel(ctx)
   215  	c.cancel = cancel
   216  	c.sink, err = c.newSink(cancelCtx)
   217  	if err != nil {
   218  		return errors.Trace(err)
   219  	}
   220  	err = c.sink.Initialize(cancelCtx, c.schema.SinkTableInfos())
   221  	if err != nil {
   222  		return errors.Trace(err)
   223  	}
   224  	// Since we wait for checkpoint == ddlJob.FinishTs before executing the DDL,
   225  	// when there is a recovery, there is no guarantee that the DDL at the checkpoint
   226  	// has been executed. So we need to start the DDL puller from (checkpoint-1).
   227  	c.ddlPuller, err = c.newDDLPuller(cancelCtx, checkpointTs-1)
   228  	if err != nil {
   229  		return errors.Trace(err)
   230  	}
   231  	c.wg.Add(1)
   232  	go func() {
   233  		defer c.wg.Done()
   234  		ctx.Throw(c.ddlPuller.Run(cancelCtx))
   235  	}()
   236  
   237  	// init metrics
   238  	c.metricsChangefeedCheckpointTsGauge = changefeedCheckpointTsGauge.WithLabelValues(c.id)
   239  	c.metricsChangefeedCheckpointTsLagGauge = changefeedCheckpointTsLagGauge.WithLabelValues(c.id)
   240  	c.initialized = true
   241  	return nil
   242  }
   243  
   244  func (c *changefeed) releaseResources() {
   245  	if !c.initialized {
   246  		return
   247  	}
   248  	log.Info("close changefeed", zap.String("changefeed", c.state.ID),
   249  		zap.Stringer("info", c.state.Info))
   250  	c.cancel()
   251  	c.cancel = func() {}
   252  	c.ddlPuller.Close()
   253  	c.schema = nil
   254  	ctx, cancel := context.WithCancel(context.Background())
   255  	cancel()
   256  	// We don't need to wait sink Close, pass a canceled context is ok
   257  	if err := c.sink.Close(ctx); err != nil {
   258  		log.Warn("Closing sink failed in Owner", zap.String("changefeedID", c.state.ID), zap.Error(err))
   259  	}
   260  	c.wg.Wait()
   261  	changefeedCheckpointTsGauge.DeleteLabelValues(c.id)
   262  	changefeedCheckpointTsLagGauge.DeleteLabelValues(c.id)
   263  	c.metricsChangefeedCheckpointTsGauge = nil
   264  	c.metricsChangefeedCheckpointTsLagGauge = nil
   265  	c.initialized = false
   266  }
   267  
   268  // preflightCheck makes sure that the metadata in Etcd is complete enough to run the tick.
   269  // If the metadata is not complete, such as when the ChangeFeedStatus is nil,
   270  // this function will reconstruct the lost metadata and skip this tick.
   271  func (c *changefeed) preflightCheck(captures map[model.CaptureID]*model.CaptureInfo) (ok bool) {
   272  	ok = true
   273  	if c.state.Status == nil {
   274  		c.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) {
   275  			if status == nil {
   276  				status = &model.ChangeFeedStatus{
   277  					// the changefeed status is nil when the changefeed is just created.
   278  					ResolvedTs:   c.state.Info.StartTs,
   279  					CheckpointTs: c.state.Info.StartTs,
   280  					AdminJobType: model.AdminNone,
   281  				}
   282  				return status, true, nil
   283  			}
   284  			return status, false, nil
   285  		})
   286  		ok = false
   287  	}
   288  	for captureID := range captures {
   289  		if _, exist := c.state.TaskStatuses[captureID]; !exist {
   290  			c.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) {
   291  				if status == nil {
   292  					status = new(model.TaskStatus)
   293  					return status, true, nil
   294  				}
   295  				return status, false, nil
   296  			})
   297  			ok = false
   298  		}
   299  	}
   300  	for captureID := range c.state.TaskStatuses {
   301  		if _, exist := captures[captureID]; !exist {
   302  			c.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) {
   303  				return nil, status != nil, nil
   304  			})
   305  			ok = false
   306  		}
   307  	}
   308  
   309  	for captureID := range c.state.TaskPositions {
   310  		if _, exist := captures[captureID]; !exist {
   311  			c.state.PatchTaskPosition(captureID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) {
   312  				return nil, position != nil, nil
   313  			})
   314  			ok = false
   315  		}
   316  	}
   317  	for captureID := range c.state.Workloads {
   318  		if _, exist := captures[captureID]; !exist {
   319  			c.state.PatchTaskWorkload(captureID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) {
   320  				return nil, workload != nil, nil
   321  			})
   322  			ok = false
   323  		}
   324  	}
   325  	return
   326  }
   327  
   328  func (c *changefeed) handleBarrier(ctx cdcContext.Context) (uint64, error) {
   329  	barrierTp, barrierTs := c.barriers.Min()
   330  	blocked := (barrierTs == c.state.Status.CheckpointTs) && (barrierTs == c.state.Status.ResolvedTs)
   331  	switch barrierTp {
   332  	case ddlJobBarrier:
   333  		ddlResolvedTs, ddlJob := c.ddlPuller.FrontDDL()
   334  		if ddlJob == nil || ddlResolvedTs != barrierTs {
   335  			if ddlResolvedTs < barrierTs {
   336  				return barrierTs, nil
   337  			}
   338  			c.barriers.Update(ddlJobBarrier, ddlResolvedTs)
   339  			return barrierTs, nil
   340  		}
   341  		if !blocked {
   342  			return barrierTs, nil
   343  		}
   344  		done, err := c.asyncExecDDL(ctx, ddlJob)
   345  		if err != nil {
   346  			return 0, errors.Trace(err)
   347  		}
   348  		if !done {
   349  			return barrierTs, nil
   350  		}
   351  		c.ddlPuller.PopFrontDDL()
   352  		newDDLResolvedTs, _ := c.ddlPuller.FrontDDL()
   353  		c.barriers.Update(ddlJobBarrier, newDDLResolvedTs)
   354  
   355  	case syncPointBarrier:
   356  		if !blocked {
   357  			return barrierTs, nil
   358  		}
   359  		nextSyncPointTs := oracle.GoTimeToTS(oracle.GetTimeFromTS(barrierTs).Add(c.state.Info.SyncPointInterval))
   360  		if err := c.sink.SinkSyncpoint(ctx, barrierTs); err != nil {
   361  			return 0, errors.Trace(err)
   362  		}
   363  		c.barriers.Update(syncPointBarrier, nextSyncPointTs)
   364  
   365  	case finishBarrier:
   366  		if !blocked {
   367  			return barrierTs, nil
   368  		}
   369  		c.feedStateManager.MarkFinished()
   370  	default:
   371  		log.Panic("Unknown barrier type", zap.Int("barrier type", int(barrierTp)))
   372  	}
   373  	return barrierTs, nil
   374  }
   375  
   376  func (c *changefeed) asyncExecDDL(ctx cdcContext.Context, job *timodel.Job) (done bool, err error) {
   377  	if job.BinlogInfo == nil {
   378  		log.Warn("ignore the invalid DDL job", zap.Reflect("job", job))
   379  		return true, nil
   380  	}
   381  	cyclicConfig := c.state.Info.Config.Cyclic
   382  	if cyclicConfig.IsEnabled() && !cyclicConfig.SyncDDL {
   383  		return true, nil
   384  	}
   385  	if c.ddlEventCache == nil || c.ddlEventCache.CommitTs != job.BinlogInfo.FinishedTS {
   386  		ddlEvent, err := c.schema.BuildDDLEvent(job)
   387  		if err != nil {
   388  			return false, errors.Trace(err)
   389  		}
   390  		err = c.schema.HandleDDL(job)
   391  		if err != nil {
   392  			return false, errors.Trace(err)
   393  		}
   394  		ddlEvent.Query = binloginfo.AddSpecialComment(ddlEvent.Query)
   395  		c.ddlEventCache = ddlEvent
   396  	}
   397  	if job.BinlogInfo.TableInfo != nil && c.schema.IsIneligibleTableID(job.BinlogInfo.TableInfo.ID) {
   398  		log.Warn("ignore the DDL job of ineligible table", zap.Reflect("job", job))
   399  		return true, nil
   400  	}
   401  	done, err = c.sink.EmitDDLEvent(ctx, c.ddlEventCache)
   402  	if err != nil {
   403  		return false, err
   404  	}
   405  	if done {
   406  		c.ddlEventCache = nil
   407  	}
   408  	return done, nil
   409  }
   410  
   411  func (c *changefeed) updateStatus(barrierTs model.Ts) {
   412  	resolvedTs := barrierTs
   413  	for _, position := range c.state.TaskPositions {
   414  		if resolvedTs > position.ResolvedTs {
   415  			resolvedTs = position.ResolvedTs
   416  		}
   417  	}
   418  	for _, taskStatus := range c.state.TaskStatuses {
   419  		for _, opt := range taskStatus.Operation {
   420  			if resolvedTs > opt.BoundaryTs {
   421  				resolvedTs = opt.BoundaryTs
   422  			}
   423  		}
   424  	}
   425  	checkpointTs := resolvedTs
   426  	for _, position := range c.state.TaskPositions {
   427  		if checkpointTs > position.CheckPointTs {
   428  			checkpointTs = position.CheckPointTs
   429  		}
   430  	}
   431  	c.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) {
   432  		changed := false
   433  		if status.ResolvedTs != resolvedTs {
   434  			status.ResolvedTs = resolvedTs
   435  			changed = true
   436  		}
   437  		if status.CheckpointTs != checkpointTs {
   438  			status.CheckpointTs = checkpointTs
   439  			changed = true
   440  		}
   441  		return status, changed, nil
   442  	})
   443  
   444  	phyTs := oracle.ExtractPhysical(checkpointTs)
   445  	c.metricsChangefeedCheckpointTsGauge.Set(float64(phyTs))
   446  	// It is more accurate to get tso from PD, but in most cases since we have
   447  	// deployed NTP service, a little bias is acceptable here.
   448  	c.metricsChangefeedCheckpointTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3)
   449  }
   450  
   451  func (c *changefeed) Close() {
   452  	c.releaseResources()
   453  }