github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/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 mq
    15  
    16  import (
    17  	"context"
    18  	"time"
    19  
    20  	"github.com/pingcap/errors"
    21  	"github.com/pingcap/log"
    22  	"github.com/pingcap/tiflow/cdc/model"
    23  	"github.com/pingcap/tiflow/cdc/sink/dmlsink"
    24  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dmlproducer"
    25  	"github.com/pingcap/tiflow/cdc/sink/metrics"
    26  	"github.com/pingcap/tiflow/cdc/sink/metrics/mq"
    27  	"github.com/pingcap/tiflow/cdc/sink/tablesink/state"
    28  	"github.com/pingcap/tiflow/pkg/chann"
    29  	"github.com/pingcap/tiflow/pkg/config"
    30  	"github.com/pingcap/tiflow/pkg/sink/codec"
    31  	"github.com/prometheus/client_golang/prometheus"
    32  	"go.uber.org/zap"
    33  	"golang.org/x/sync/errgroup"
    34  )
    35  
    36  const (
    37  	// batchSize is the maximum size of the number of messages in a batch.
    38  	batchSize = 2048
    39  	// batchInterval is the interval of the worker to collect a batch of messages.
    40  	// It shouldn't be too large, otherwise it will lead to a high latency.
    41  	batchInterval = 15 * time.Millisecond
    42  )
    43  
    44  // mqEvent is the event of the mq worker.
    45  // It carries the topic and partition information of the message.
    46  type mqEvent struct {
    47  	key      model.TopicPartitionKey
    48  	rowEvent *dmlsink.RowChangeCallbackableEvent
    49  }
    50  
    51  // worker will send messages to the DML producer on a batch basis.
    52  type worker struct {
    53  	// changeFeedID indicates this sink belongs to which processor(changefeed).
    54  	changeFeedID model.ChangeFeedID
    55  	// protocol indicates the protocol used by this sink.
    56  	protocol config.Protocol
    57  	// msgChan caches the messages to be sent.
    58  	// It is an unbounded channel.
    59  	msgChan *chann.DrainableChann[mqEvent]
    60  	// ticker used to force flush the batched messages when the interval is reached.
    61  	ticker *time.Ticker
    62  
    63  	encoderGroup codec.EncoderGroup
    64  
    65  	// producer is used to send the messages to the Kafka broker.
    66  	producer dmlproducer.DMLProducer
    67  
    68  	// metricMQWorkerSendMessageDuration tracks the time duration cost on send messages.
    69  	metricMQWorkerSendMessageDuration prometheus.Observer
    70  	// metricMQWorkerBatchSize tracks each batch's size.
    71  	metricMQWorkerBatchSize prometheus.Observer
    72  	// metricMQWorkerBatchDuration tracks the time duration cost on batch messages.
    73  	metricMQWorkerBatchDuration prometheus.Observer
    74  	// statistics is used to record DML metrics.
    75  	statistics *metrics.Statistics
    76  }
    77  
    78  // newWorker creates a new flush worker.
    79  func newWorker(
    80  	id model.ChangeFeedID,
    81  	protocol config.Protocol,
    82  	producer dmlproducer.DMLProducer,
    83  	encoderGroup codec.EncoderGroup,
    84  	statistics *metrics.Statistics,
    85  ) *worker {
    86  	w := &worker{
    87  		changeFeedID:                      id,
    88  		protocol:                          protocol,
    89  		msgChan:                           chann.NewAutoDrainChann[mqEvent](),
    90  		ticker:                            time.NewTicker(batchInterval),
    91  		encoderGroup:                      encoderGroup,
    92  		producer:                          producer,
    93  		metricMQWorkerSendMessageDuration: mq.WorkerSendMessageDuration.WithLabelValues(id.Namespace, id.ID),
    94  		metricMQWorkerBatchSize:           mq.WorkerBatchSize.WithLabelValues(id.Namespace, id.ID),
    95  		metricMQWorkerBatchDuration:       mq.WorkerBatchDuration.WithLabelValues(id.Namespace, id.ID),
    96  		statistics:                        statistics,
    97  	}
    98  
    99  	return w
   100  }
   101  
   102  // run starts a loop that keeps collecting, sorting and sending messages
   103  // until it encounters an error or is interrupted.
   104  func (w *worker) run(ctx context.Context) (retErr error) {
   105  	defer func() {
   106  		w.ticker.Stop()
   107  		log.Info("MQ sink worker exited", zap.Error(retErr),
   108  			zap.String("namespace", w.changeFeedID.Namespace),
   109  			zap.String("changefeed", w.changeFeedID.ID),
   110  			zap.String("protocol", w.protocol.String()),
   111  		)
   112  	}()
   113  
   114  	g, ctx := errgroup.WithContext(ctx)
   115  	g.Go(func() error {
   116  		return w.encoderGroup.Run(ctx)
   117  	})
   118  	g.Go(func() error {
   119  		if w.protocol.IsBatchEncode() {
   120  			return w.batchEncodeRun(ctx)
   121  		}
   122  		return w.nonBatchEncodeRun(ctx)
   123  	})
   124  	g.Go(func() error {
   125  		return w.sendMessages(ctx)
   126  	})
   127  	return g.Wait()
   128  }
   129  
   130  // nonBatchEncodeRun add events to the encoder group immediately.
   131  func (w *worker) nonBatchEncodeRun(ctx context.Context) error {
   132  	log.Info("MQ sink non batch worker started",
   133  		zap.String("namespace", w.changeFeedID.Namespace),
   134  		zap.String("changefeed", w.changeFeedID.ID),
   135  		zap.String("protocol", w.protocol.String()),
   136  	)
   137  	for {
   138  		select {
   139  		case <-ctx.Done():
   140  			return errors.Trace(ctx.Err())
   141  		case event, ok := <-w.msgChan.Out():
   142  			if !ok {
   143  				log.Warn("MQ sink flush worker channel closed",
   144  					zap.String("namespace", w.changeFeedID.Namespace),
   145  					zap.String("changefeed", w.changeFeedID.ID))
   146  				return nil
   147  			}
   148  			if event.rowEvent.GetTableSinkState() != state.TableSinkSinking {
   149  				event.rowEvent.Callback()
   150  				log.Debug("Skip event of stopped table",
   151  					zap.String("namespace", w.changeFeedID.Namespace),
   152  					zap.String("changefeed", w.changeFeedID.ID),
   153  					zap.Any("event", event))
   154  				continue
   155  			}
   156  			if err := w.encoderGroup.AddEvents(
   157  				ctx,
   158  				event.key,
   159  				event.rowEvent); err != nil {
   160  				return errors.Trace(err)
   161  			}
   162  		}
   163  	}
   164  }
   165  
   166  // batchEncodeRun collect messages into batch and add them to the encoder group.
   167  func (w *worker) batchEncodeRun(ctx context.Context) (retErr error) {
   168  	log.Info("MQ sink batch worker started",
   169  		zap.String("namespace", w.changeFeedID.Namespace),
   170  		zap.String("changefeed", w.changeFeedID.ID),
   171  		zap.String("protocol", w.protocol.String()),
   172  	)
   173  
   174  	msgsBuf := make([]mqEvent, batchSize)
   175  	for {
   176  		start := time.Now()
   177  		msgCount, err := w.batch(ctx, msgsBuf, batchInterval)
   178  		if err != nil {
   179  			return errors.Trace(err)
   180  		}
   181  		if msgCount == 0 {
   182  			continue
   183  		}
   184  
   185  		w.metricMQWorkerBatchSize.Observe(float64(msgCount))
   186  		w.metricMQWorkerBatchDuration.Observe(time.Since(start).Seconds())
   187  
   188  		msgs := msgsBuf[:msgCount]
   189  		// Group messages by its TopicPartitionKey before adding them to the encoder group.
   190  		groupedMsgs := w.group(msgs)
   191  		for key, msg := range groupedMsgs {
   192  			if err := w.encoderGroup.AddEvents(ctx, key, msg...); err != nil {
   193  				return errors.Trace(err)
   194  			}
   195  		}
   196  	}
   197  }
   198  
   199  // batch collects a batch of messages from w.msgChan into buffer.
   200  // It returns the number of messages collected.
   201  // Note: It will block until at least one message is received.
   202  func (w *worker) batch(
   203  	ctx context.Context, buffer []mqEvent, flushInterval time.Duration,
   204  ) (int, error) {
   205  	msgCount := 0
   206  	maxBatchSize := len(buffer)
   207  	// We need to receive at least one message or be interrupted,
   208  	// otherwise it will lead to idling.
   209  	select {
   210  	case <-ctx.Done():
   211  		return msgCount, ctx.Err()
   212  	case msg, ok := <-w.msgChan.Out():
   213  		if !ok {
   214  			log.Warn("MQ sink flush worker channel closed")
   215  			return msgCount, nil
   216  		}
   217  		if msg.rowEvent != nil {
   218  			w.statistics.ObserveRows(msg.rowEvent.Event)
   219  			buffer[msgCount] = msg
   220  			msgCount++
   221  		}
   222  	}
   223  
   224  	// Reset the ticker to start a new batching.
   225  	// We need to stop batching when the interval is reached.
   226  	w.ticker.Reset(flushInterval)
   227  	for {
   228  		select {
   229  		case <-ctx.Done():
   230  			return msgCount, ctx.Err()
   231  		case msg, ok := <-w.msgChan.Out():
   232  			if !ok {
   233  				log.Warn("MQ sink flush worker channel closed")
   234  				return msgCount, nil
   235  			}
   236  
   237  			if msg.rowEvent != nil {
   238  				w.statistics.ObserveRows(msg.rowEvent.Event)
   239  				buffer[msgCount] = msg
   240  				msgCount++
   241  			}
   242  
   243  			if msgCount >= maxBatchSize {
   244  				return msgCount, nil
   245  			}
   246  		case <-w.ticker.C:
   247  			return msgCount, nil
   248  		}
   249  	}
   250  }
   251  
   252  // group groups messages by its key.
   253  func (w *worker) group(
   254  	msgs []mqEvent,
   255  ) map[model.TopicPartitionKey][]*dmlsink.RowChangeCallbackableEvent {
   256  	groupedMsgs := make(map[model.TopicPartitionKey][]*dmlsink.RowChangeCallbackableEvent)
   257  	for _, msg := range msgs {
   258  		// Skip this event when the table is stopping.
   259  		if msg.rowEvent.GetTableSinkState() != state.TableSinkSinking {
   260  			msg.rowEvent.Callback()
   261  			log.Debug("Skip event of stopped table", zap.Any("event", msg.rowEvent))
   262  			continue
   263  		}
   264  		if _, ok := groupedMsgs[msg.key]; !ok {
   265  			groupedMsgs[msg.key] = make([]*dmlsink.RowChangeCallbackableEvent, 0)
   266  		}
   267  		groupedMsgs[msg.key] = append(groupedMsgs[msg.key], msg.rowEvent)
   268  	}
   269  	return groupedMsgs
   270  }
   271  
   272  func (w *worker) sendMessages(ctx context.Context) error {
   273  	ticker := time.NewTicker(15 * time.Second)
   274  	metric := codec.EncoderGroupOutputChanSizeGauge.
   275  		WithLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
   276  	defer func() {
   277  		ticker.Stop()
   278  		codec.EncoderGroupOutputChanSizeGauge.
   279  			DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
   280  	}()
   281  
   282  	var err error
   283  	outCh := w.encoderGroup.Output()
   284  	for {
   285  		select {
   286  		case <-ctx.Done():
   287  			return errors.Trace(ctx.Err())
   288  		case <-ticker.C:
   289  			metric.Set(float64(len(outCh)))
   290  		case future, ok := <-outCh:
   291  			if !ok {
   292  				log.Warn("MQ sink encoder's output channel closed",
   293  					zap.String("namespace", w.changeFeedID.Namespace),
   294  					zap.String("changefeed", w.changeFeedID.ID))
   295  				return nil
   296  			}
   297  			if err = future.Ready(ctx); err != nil {
   298  				return errors.Trace(err)
   299  			}
   300  			for _, message := range future.Messages {
   301  				start := time.Now()
   302  				if err = w.statistics.RecordBatchExecution(func() (int, int64, error) {
   303  					message.SetPartitionKey(future.Key.PartitionKey)
   304  					if err := w.producer.AsyncSendMessage(
   305  						ctx,
   306  						future.Key.Topic,
   307  						future.Key.Partition,
   308  						message); err != nil {
   309  						return 0, 0, err
   310  					}
   311  					return message.GetRowsCount(), int64(message.Length()), nil
   312  				}); err != nil {
   313  					return err
   314  				}
   315  				w.metricMQWorkerSendMessageDuration.Observe(time.Since(start).Seconds())
   316  			}
   317  		}
   318  	}
   319  }
   320  
   321  func (w *worker) close() {
   322  	w.msgChan.CloseAndDrain()
   323  	w.producer.Close()
   324  	mq.WorkerSendMessageDuration.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
   325  	mq.WorkerBatchSize.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
   326  	mq.WorkerBatchDuration.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
   327  }