github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/processor/pipeline/sorter.go (about)

     1  // Copyright 2020 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 pipeline
    15  
    16  import (
    17  	"context"
    18  	"time"
    19  
    20  	"github.com/pingcap/errors"
    21  	"github.com/pingcap/failpoint"
    22  	"github.com/pingcap/log"
    23  	"github.com/pingcap/ticdc/cdc/entry"
    24  	"github.com/pingcap/ticdc/cdc/model"
    25  	"github.com/pingcap/ticdc/cdc/puller"
    26  	psorter "github.com/pingcap/ticdc/cdc/puller/sorter"
    27  	cerror "github.com/pingcap/ticdc/pkg/errors"
    28  	"github.com/pingcap/ticdc/pkg/pipeline"
    29  	"go.uber.org/zap"
    30  	"golang.org/x/sync/errgroup"
    31  )
    32  
    33  const (
    34  	flushMemoryMetricsDuration = time.Second * 5
    35  )
    36  
    37  type sorterNode struct {
    38  	sorter puller.EventSorter
    39  
    40  	tableID   model.TableID
    41  	tableName string // quoted schema and table, used in metircs only
    42  
    43  	// for per-table flow control
    44  	flowController tableFlowController
    45  
    46  	mounter entry.Mounter
    47  
    48  	wg     errgroup.Group
    49  	cancel context.CancelFunc
    50  }
    51  
    52  func newSorterNode(tableName string, tableID model.TableID, flowController tableFlowController, mounter entry.Mounter) pipeline.Node {
    53  	return &sorterNode{
    54  		tableName:      tableName,
    55  		tableID:        tableID,
    56  		flowController: flowController,
    57  		mounter:        mounter,
    58  	}
    59  }
    60  
    61  func (n *sorterNode) Init(ctx pipeline.NodeContext) error {
    62  	stdCtx, cancel := context.WithCancel(ctx)
    63  	n.cancel = cancel
    64  	var sorter puller.EventSorter
    65  	sortEngine := ctx.ChangefeedVars().Info.Engine
    66  	switch sortEngine {
    67  	case model.SortInMemory:
    68  		sorter = puller.NewEntrySorter()
    69  	case model.SortUnified, model.SortInFile /* `file` becomes an alias of `unified` for backward compatibility */ :
    70  		if sortEngine == model.SortInFile {
    71  			log.Warn("File sorter is obsolete and replaced by unified sorter. Please revise your changefeed settings",
    72  				zap.String("changefeed-id", ctx.ChangefeedVars().ID), zap.String("table-name", n.tableName))
    73  		}
    74  		sortDir := ctx.ChangefeedVars().Info.SortDir
    75  		err := psorter.UnifiedSorterCheckDir(sortDir)
    76  		if err != nil {
    77  			return errors.Trace(err)
    78  		}
    79  		sorter, err = psorter.NewUnifiedSorter(sortDir, ctx.ChangefeedVars().ID, n.tableName, n.tableID, ctx.GlobalVars().CaptureInfo.AdvertiseAddr)
    80  		if err != nil {
    81  			return errors.Trace(err)
    82  		}
    83  	default:
    84  		return cerror.ErrUnknownSortEngine.GenWithStackByArgs(sortEngine)
    85  	}
    86  	failpoint.Inject("ProcessorAddTableError", func() {
    87  		failpoint.Return(errors.New("processor add table injected error"))
    88  	})
    89  	n.wg.Go(func() error {
    90  		ctx.Throw(errors.Trace(sorter.Run(stdCtx)))
    91  		return nil
    92  	})
    93  	n.wg.Go(func() error {
    94  		// Since the flowController is implemented by `Cond`, it is not cancelable
    95  		// by a context. We need to listen on cancellation and aborts the flowController
    96  		// manually.
    97  		<-stdCtx.Done()
    98  		n.flowController.Abort()
    99  		return nil
   100  	})
   101  	n.wg.Go(func() error {
   102  		lastSentResolvedTs := uint64(0)
   103  		lastSendResolvedTsTime := time.Now() // the time at which we last sent a resolved-ts.
   104  		lastCRTs := uint64(0)                // the commit-ts of the last row changed we sent.
   105  
   106  		metricsTableMemoryHistogram := tableMemoryHistogram.WithLabelValues(ctx.ChangefeedVars().ID, ctx.GlobalVars().CaptureInfo.AdvertiseAddr)
   107  		metricsTicker := time.NewTicker(flushMemoryMetricsDuration)
   108  		defer metricsTicker.Stop()
   109  
   110  		for {
   111  			select {
   112  			case <-stdCtx.Done():
   113  				return nil
   114  			case <-metricsTicker.C:
   115  				metricsTableMemoryHistogram.Observe(float64(n.flowController.GetConsumption()))
   116  			case msg, ok := <-sorter.Output():
   117  				if !ok {
   118  					// sorter output channel closed
   119  					return nil
   120  				}
   121  				if msg == nil || msg.RawKV == nil {
   122  					log.Panic("unexpected empty msg", zap.Reflect("msg", msg))
   123  				}
   124  				if msg.RawKV.OpType != model.OpTypeResolved {
   125  					size := uint64(msg.RawKV.ApproximateSize())
   126  					commitTs := msg.CRTs
   127  					// We interpolate a resolved-ts if none has been sent for some time.
   128  					if time.Since(lastSendResolvedTsTime) > resolvedTsInterpolateInterval {
   129  						// checks the condition: cur_event_commit_ts > prev_event_commit_ts > last_resolved_ts
   130  						// If this is true, it implies that (1) the last transaction has finished, and we are processing
   131  						// the first event in a new transaction, (2) a resolved-ts is safe to be sent, but it has not yet.
   132  						// This means that we can interpolate prev_event_commit_ts as a resolved-ts, improving the frequency
   133  						// at which the sink flushes.
   134  						if lastCRTs > lastSentResolvedTs && commitTs > lastCRTs {
   135  							lastSentResolvedTs = lastCRTs
   136  							lastSendResolvedTsTime = time.Now()
   137  							ctx.SendToNextNode(pipeline.PolymorphicEventMessage(model.NewResolvedPolymorphicEvent(0, lastCRTs)))
   138  						}
   139  					}
   140  					// NOTE we allow the quota to be exceeded if blocking means interrupting a transaction.
   141  					// Otherwise the pipeline would deadlock.
   142  					err := n.flowController.Consume(commitTs, size, func() error {
   143  						if lastCRTs > lastSentResolvedTs {
   144  							// If we are blocking, we send a Resolved Event here to elicit a sink-flush.
   145  							// Not sending a Resolved Event here will very likely deadlock the pipeline.
   146  							lastSentResolvedTs = lastCRTs
   147  							lastSendResolvedTsTime = time.Now()
   148  							ctx.SendToNextNode(pipeline.PolymorphicEventMessage(model.NewResolvedPolymorphicEvent(0, lastCRTs)))
   149  						}
   150  						return nil
   151  					})
   152  					if err != nil {
   153  						if cerror.ErrFlowControllerAborted.Equal(err) {
   154  							log.Info("flow control cancelled for table",
   155  								zap.Int64("tableID", n.tableID),
   156  								zap.String("tableName", n.tableName))
   157  						} else {
   158  							ctx.Throw(err)
   159  						}
   160  						return nil
   161  					}
   162  					lastCRTs = commitTs
   163  
   164  					// DESIGN NOTE: We send the messages to the mounter in this separate goroutine to prevent
   165  					// blocking the whole pipeline.
   166  					msg.SetUpFinishedChan()
   167  					select {
   168  					case <-ctx.Done():
   169  						return nil
   170  					case n.mounter.Input() <- msg:
   171  					}
   172  				} else {
   173  					// handle OpTypeResolved
   174  					if msg.CRTs < lastSentResolvedTs {
   175  						continue
   176  					}
   177  					lastSentResolvedTs = msg.CRTs
   178  					lastSendResolvedTsTime = time.Now()
   179  				}
   180  				ctx.SendToNextNode(pipeline.PolymorphicEventMessage(msg))
   181  			}
   182  		}
   183  	})
   184  	n.sorter = sorter
   185  	return nil
   186  }
   187  
   188  // Receive receives the message from the previous node
   189  func (n *sorterNode) Receive(ctx pipeline.NodeContext) error {
   190  	msg := ctx.Message()
   191  	switch msg.Tp {
   192  	case pipeline.MessageTypePolymorphicEvent:
   193  		n.sorter.AddEntry(ctx, msg.PolymorphicEvent)
   194  	default:
   195  		ctx.SendToNextNode(msg)
   196  	}
   197  	return nil
   198  }
   199  
   200  func (n *sorterNode) Destroy(ctx pipeline.NodeContext) error {
   201  	defer tableMemoryHistogram.DeleteLabelValues(ctx.ChangefeedVars().ID, ctx.GlobalVars().CaptureInfo.AdvertiseAddr)
   202  	n.cancel()
   203  	return n.wg.Wait()
   204  }