github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sinkmanager/table_sink_worker.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 sinkmanager
    15  
    16  import (
    17  	"context"
    18  	"sync/atomic"
    19  	"time"
    20  
    21  	"github.com/pingcap/errors"
    22  	"github.com/pingcap/failpoint"
    23  	"github.com/pingcap/log"
    24  	"github.com/pingcap/tiflow/cdc/model"
    25  	"github.com/pingcap/tiflow/cdc/processor/memquota"
    26  	"github.com/pingcap/tiflow/cdc/processor/sourcemanager"
    27  	"github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter"
    28  	"github.com/pingcap/tiflow/cdc/sink/tablesink"
    29  	"github.com/prometheus/client_golang/prometheus"
    30  	"github.com/tikv/client-go/v2/oracle"
    31  	"go.uber.org/zap"
    32  )
    33  
    34  // batchID is used to advance table sink with a given CommitTs, even if not all
    35  // transactions with the same CommitTs are collected, regardless of whether splitTxn
    36  // is enabled or not. We split transactions with the same CommitTs even if splitTxn
    37  // is false, and it won't break transaction atomicity to downstream.
    38  // NOTICE:
    39  // batchID is used to distinguish different batches of the same transaction.
    40  // We need to use a global variable because the same commit ts event may be
    41  // processed at different times.
    42  // For example:
    43  //  1. The commit ts is 1000, and the start ts is 998.
    44  //  2. Keep fetching events and flush them to the sink with batch ID 1.
    45  //  3. Because we don't have enough memory quota, we need to flush the events
    46  //     and wait for the next round of processing.
    47  //  4. The next round of processing starts at commit ts 1000, and the start ts
    48  //     is 999.
    49  //  5. The batch ID restarts from 1, and the commit ts still is 1000.
    50  //  6. We flush all the events with commit ts 1000 and batch ID 1 to the sink.
    51  //  7. We release the memory quota of the events earlier because the current
    52  //     round of processing is not finished.
    53  //
    54  // Therefore, we must use a global variable to ensure that the batch ID is
    55  // monotonically increasing.
    56  // We share this variable for all workers, it is OK that the batch ID is not
    57  // strictly increasing one by one.
    58  var batchID atomic.Uint64
    59  
    60  type sinkWorker struct {
    61  	changefeedID  model.ChangeFeedID
    62  	sourceManager *sourcemanager.SourceManager
    63  	sinkMemQuota  *memquota.MemQuota
    64  	// splitTxn indicates whether to split the transaction into multiple batches.
    65  	splitTxn bool
    66  
    67  	// Metrics.
    68  	metricOutputEventCountKV prometheus.Counter
    69  }
    70  
    71  // newSinkWorker creates a new sink worker.
    72  func newSinkWorker(
    73  	changefeedID model.ChangeFeedID,
    74  	sourceManager *sourcemanager.SourceManager,
    75  	sinkQuota *memquota.MemQuota,
    76  	splitTxn bool,
    77  ) *sinkWorker {
    78  	return &sinkWorker{
    79  		changefeedID:  changefeedID,
    80  		sourceManager: sourceManager,
    81  		sinkMemQuota:  sinkQuota,
    82  		splitTxn:      splitTxn,
    83  
    84  		metricOutputEventCountKV: outputEventCount.WithLabelValues(changefeedID.Namespace, changefeedID.ID, "kv"),
    85  	}
    86  }
    87  
    88  func (w *sinkWorker) handleTasks(ctx context.Context, taskChan <-chan *sinkTask) error {
    89  	failpoint.Inject("SinkWorkerTaskHandlePause", func() { <-ctx.Done() })
    90  	for {
    91  		select {
    92  		case <-ctx.Done():
    93  			return ctx.Err()
    94  		case task := <-taskChan:
    95  			err := w.handleTask(ctx, task)
    96  			failpoint.Inject("SinkWorkerTaskError", func() {
    97  				err = errors.New("SinkWorkerTaskError")
    98  			})
    99  			if err != nil {
   100  				return err
   101  			}
   102  		}
   103  	}
   104  }
   105  
   106  func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr error) {
   107  	// We need to use a new batch ID for each task.
   108  	batchID.Add(1)
   109  	advancer := newTableSinkAdvancer(task, w.splitTxn, w.sinkMemQuota, requestMemSize)
   110  	// The task is finished and some required memory isn't used.
   111  	defer advancer.cleanup()
   112  
   113  	lowerBound, upperBound := validateAndAdjustBound(
   114  		w.changefeedID,
   115  		&task.span,
   116  		task.lowerBound,
   117  		task.getUpperBound(task.tableSink.getUpperBoundTs()))
   118  	advancer.lastPos = lowerBound.Prev()
   119  
   120  	allEventCount := 0
   121  
   122  	callbackIsPerformed := false
   123  	performCallback := func(pos sorter.Position) {
   124  		if !callbackIsPerformed {
   125  			task.callback(pos)
   126  			callbackIsPerformed = true
   127  		}
   128  	}
   129  
   130  	defer func() {
   131  		// Prepare some information for stale table range cleaning.
   132  		task.tableSink.updateRangeEventCounts(newRangeEventCount(advancer.lastPos, allEventCount))
   133  
   134  		// Collect metrics.
   135  		w.metricOutputEventCountKV.Add(float64(allEventCount))
   136  
   137  		log.Debug("Sink task finished",
   138  			zap.String("namespace", w.changefeedID.Namespace),
   139  			zap.String("changefeed", w.changefeedID.ID),
   140  			zap.Stringer("span", &task.span),
   141  			zap.Any("lowerBound", lowerBound),
   142  			zap.Any("upperBound", upperBound),
   143  			zap.Bool("splitTxn", w.splitTxn),
   144  			zap.Int("receivedEvents", allEventCount),
   145  			zap.Any("lastPos", advancer.lastPos),
   146  			zap.Float64("lag", time.Since(oracle.GetTimeFromTS(advancer.lastPos.CommitTs)).Seconds()),
   147  			zap.Error(finalErr))
   148  
   149  		// Otherwise we can't ensure all events before `lastPos` are emitted.
   150  		if finalErr == nil {
   151  			performCallback(advancer.lastPos)
   152  		} else {
   153  			switch errors.Cause(finalErr).(type) {
   154  			// If it's a warning, close the table sink and wait all pending
   155  			// events have been reported. Then we can continue the table
   156  			// at the checkpoint position.
   157  			case tablesink.SinkInternalError:
   158  				// After the table sink is cleared all pending events are sent out or dropped.
   159  				// So we can re-add the table into sinkMemQuota.
   160  				w.sinkMemQuota.ClearTable(task.tableSink.span)
   161  				performCallback(advancer.lastPos)
   162  				finalErr = nil
   163  			default:
   164  			}
   165  		}
   166  	}()
   167  
   168  	// lowerBound and upperBound are both closed intervals.
   169  	iter := w.sourceManager.FetchByTable(task.span, lowerBound, upperBound, w.sinkMemQuota)
   170  	defer func() {
   171  		if err := iter.Close(); err != nil {
   172  			log.Error("sink worker fails to close iterator",
   173  				zap.String("namespace", w.changefeedID.Namespace),
   174  				zap.String("changefeed", w.changefeedID.ID),
   175  				zap.Stringer("span", &task.span),
   176  				zap.Error(err))
   177  		}
   178  	}()
   179  
   180  	// 1. We have enough memory to collect events.
   181  	// 2. The task is not canceled.
   182  	for advancer.hasEnoughMem() && !task.isCanceled() {
   183  		e, pos, err := iter.Next(ctx)
   184  		if err != nil {
   185  			return errors.Trace(err)
   186  		}
   187  
   188  		// There is no more data. It means that we finish this scan task.
   189  		if e == nil {
   190  			return advancer.finish(upperBound)
   191  		}
   192  
   193  		allEventCount += 1
   194  
   195  		// Only record the last valid position.
   196  		// If the current txn is not finished, the position is not valid.
   197  		if pos.Valid() {
   198  			advancer.lastPos = pos
   199  		}
   200  
   201  		// Meet a new commit ts, we need to emit the previous events.
   202  		advancer.tryMoveToNextTxn(e.CRTs)
   203  
   204  		// NOTICE: The event can be filtered by the event filter.
   205  		if e.Row != nil {
   206  			// For all rows, we add table replicate ts, so mysql sink can determine safe-mode.
   207  			e.Row.ReplicatingTs = task.tableSink.replicateTs
   208  			x, size := handleRowChangedEvents(w.changefeedID, task.span, e)
   209  			advancer.appendEvents(x, size)
   210  		}
   211  
   212  		if err := advancer.tryAdvanceAndAcquireMem(false, pos.Valid()); err != nil {
   213  			return errors.Trace(err)
   214  		}
   215  	}
   216  
   217  	return advancer.lastTimeAdvance()
   218  }