github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/async_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 cdc
    15  
    16  // TODO: to remove this file once new owner is enabled.
    17  
    18  import (
    19  	"context"
    20  	"sync"
    21  	"sync/atomic"
    22  	"time"
    23  
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/failpoint"
    26  	"github.com/pingcap/log"
    27  	"github.com/pingcap/ticdc/cdc/model"
    28  	"github.com/pingcap/ticdc/cdc/sink"
    29  	cdcContext "github.com/pingcap/ticdc/pkg/context"
    30  	cerror "github.com/pingcap/ticdc/pkg/errors"
    31  	"github.com/pingcap/ticdc/pkg/filter"
    32  	"go.uber.org/zap"
    33  )
    34  
    35  const (
    36  	defaultErrChSize = 1024
    37  )
    38  
    39  // AsyncSink is an async sink design for owner
    40  // The EmitCheckpointTs and EmitDDLEvent is asynchronous function for now
    41  // Other functions are still synchronization
    42  type AsyncSink interface {
    43  	Initialize(ctx cdcContext.Context, tableInfo []*model.SimpleTableInfo) error
    44  	// EmitCheckpointTs emits the checkpoint Ts to downstream data source
    45  	// this function will return after recording the checkpointTs specified in memory immediately
    46  	// and the recorded checkpointTs will be sent and updated to downstream data source every second
    47  	// return err here for the unit test TestOwnerCalcResolvedTs in owner_test
    48  	EmitCheckpointTs(ctx cdcContext.Context, ts uint64) error
    49  	// EmitDDLEvent emits DDL event asynchronously and return true if the DDL is executed
    50  	// the DDL event will be sent to another goroutine and execute to downstream
    51  	// the caller of this function can call again and again until a true returned
    52  	EmitDDLEvent(ctx cdcContext.Context, ddl *model.DDLEvent) (bool, error)
    53  	SinkSyncpoint(ctx cdcContext.Context, checkpointTs uint64) error
    54  	Close(ctx context.Context) error
    55  }
    56  
    57  type asyncSinkImpl struct {
    58  	sink           sink.Sink
    59  	syncpointStore sink.SyncpointStore
    60  
    61  	checkpointTs model.Ts
    62  
    63  	lastSyncPoint model.Ts
    64  
    65  	ddlCh         chan *model.DDLEvent
    66  	ddlFinishedTs model.Ts
    67  	ddlSentTs     model.Ts
    68  
    69  	cancel context.CancelFunc
    70  	wg     sync.WaitGroup
    71  	errCh  chan error
    72  }
    73  
    74  func newAsyncSink(ctx cdcContext.Context) (AsyncSink, error) {
    75  	ctx, cancel := cdcContext.WithCancel(ctx)
    76  	changefeedID := ctx.ChangefeedVars().ID
    77  	changefeedInfo := ctx.ChangefeedVars().Info
    78  	filter, err := filter.NewFilter(changefeedInfo.Config)
    79  	if err != nil {
    80  		return nil, errors.Trace(err)
    81  	}
    82  	errCh := make(chan error, defaultErrChSize)
    83  	s, err := sink.NewSink(ctx, changefeedID, changefeedInfo.SinkURI, filter, changefeedInfo.Config, changefeedInfo.Opts, errCh)
    84  	if err != nil {
    85  		return nil, errors.Trace(err)
    86  	}
    87  	asyncSink := &asyncSinkImpl{
    88  		sink:   s,
    89  		ddlCh:  make(chan *model.DDLEvent, 1),
    90  		errCh:  errCh,
    91  		cancel: cancel,
    92  	}
    93  
    94  	asyncSink.wg.Add(1)
    95  	go asyncSink.run(ctx)
    96  	return asyncSink, nil
    97  }
    98  
    99  func (s *asyncSinkImpl) Initialize(ctx cdcContext.Context, tableInfo []*model.SimpleTableInfo) error {
   100  	return s.sink.Initialize(ctx, tableInfo)
   101  }
   102  
   103  func (s *asyncSinkImpl) run(ctx cdcContext.Context) {
   104  	defer s.wg.Done()
   105  	// TODO make the tick duration configurable
   106  	ticker := time.NewTicker(time.Second)
   107  	defer ticker.Stop()
   108  	var lastCheckpointTs model.Ts
   109  	for {
   110  		select {
   111  		case <-ctx.Done():
   112  			return
   113  		case err := <-s.errCh:
   114  			ctx.Throw(err)
   115  			return
   116  		case <-ticker.C:
   117  			checkpointTs := atomic.LoadUint64(&s.checkpointTs)
   118  			if checkpointTs == 0 || checkpointTs <= lastCheckpointTs {
   119  				continue
   120  			}
   121  			lastCheckpointTs = checkpointTs
   122  			if err := s.sink.EmitCheckpointTs(ctx, checkpointTs); err != nil {
   123  				ctx.Throw(errors.Trace(err))
   124  				return
   125  			}
   126  		case ddl := <-s.ddlCh:
   127  			err := s.sink.EmitDDLEvent(ctx, ddl)
   128  			failpoint.Inject("InjectChangefeedAsyncDDLError", func() {
   129  				err = cerror.ErrExecDDLFailed.GenWithStack("InjectChangefeedAsyncDDLError")
   130  			})
   131  			if err == nil || cerror.ErrDDLEventIgnored.Equal(err) {
   132  				log.Info("Execute DDL succeeded", zap.String("changefeed", ctx.ChangefeedVars().ID), zap.Bool("ignored", err != nil), zap.Reflect("ddl", ddl))
   133  				atomic.StoreUint64(&s.ddlFinishedTs, ddl.CommitTs)
   134  			} else {
   135  				// If DDL executing failed, and the error can not be ignored, throw an error and pause the changefeed
   136  				log.Error("Execute DDL failed",
   137  					zap.String("ChangeFeedID", ctx.ChangefeedVars().ID),
   138  					zap.Error(err),
   139  					zap.Reflect("ddl", ddl))
   140  				ctx.Throw(errors.Trace(err))
   141  				return
   142  			}
   143  		}
   144  	}
   145  }
   146  
   147  func (s *asyncSinkImpl) EmitCheckpointTs(ctx cdcContext.Context, ts uint64) error {
   148  	atomic.StoreUint64(&s.checkpointTs, ts)
   149  	return nil
   150  }
   151  
   152  func (s *asyncSinkImpl) EmitDDLEvent(ctx cdcContext.Context, ddl *model.DDLEvent) (bool, error) {
   153  	ddlFinishedTs := atomic.LoadUint64(&s.ddlFinishedTs)
   154  	failpoint.Inject("InjectChangefeedDDLBlock", func() {
   155  		if ctx.ChangefeedVars().ID == "changefeed-ddl-block" {
   156  			log.Info("step into failpoint")
   157  			// make the func EmitDDLEvent always return false
   158  			// tests/ddl_async will fail if ddl block the owner
   159  			ddlFinishedTs = ddl.CommitTs - 10
   160  			ddl.CommitTs = s.ddlSentTs - 10
   161  		}
   162  	})
   163  	if ddl.CommitTs <= ddlFinishedTs {
   164  		// the DDL event is executed successfully, and done is true
   165  		return true, nil
   166  	}
   167  	if ddl.CommitTs <= s.ddlSentTs {
   168  		// the DDL event is executing and not finished yes, return false
   169  		return false, nil
   170  	}
   171  	select {
   172  	case <-ctx.Done():
   173  		return false, errors.Trace(ctx.Err())
   174  	case s.ddlCh <- ddl:
   175  	}
   176  	s.ddlSentTs = ddl.CommitTs
   177  	return false, nil
   178  }
   179  
   180  func (s *asyncSinkImpl) SinkSyncpoint(ctx cdcContext.Context, checkpointTs uint64) error {
   181  	if checkpointTs == s.lastSyncPoint {
   182  		return nil
   183  	}
   184  	s.lastSyncPoint = checkpointTs
   185  	// TODO implement async sink syncpoint
   186  	return s.syncpointStore.SinkSyncpoint(ctx, ctx.ChangefeedVars().ID, checkpointTs)
   187  }
   188  
   189  func (s *asyncSinkImpl) Close(ctx context.Context) (err error) {
   190  	s.cancel()
   191  	err = s.sink.Close(ctx)
   192  	if s.syncpointStore != nil {
   193  		err = s.syncpointStore.Close()
   194  	}
   195  	s.wg.Wait()
   196  	return
   197  }