github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/mq_dml_sink.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  	"sync"
    19  
    20  	"github.com/pingcap/errors"
    21  	"github.com/pingcap/failpoint"
    22  	"github.com/pingcap/log"
    23  	"github.com/pingcap/tiflow/cdc/model"
    24  	"github.com/pingcap/tiflow/cdc/sink/dmlsink"
    25  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher"
    26  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dmlproducer"
    27  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/manager"
    28  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/transformer"
    29  	"github.com/pingcap/tiflow/cdc/sink/metrics"
    30  	"github.com/pingcap/tiflow/cdc/sink/tablesink/state"
    31  	"github.com/pingcap/tiflow/pkg/config"
    32  	"github.com/pingcap/tiflow/pkg/sink"
    33  	"github.com/pingcap/tiflow/pkg/sink/codec"
    34  	"github.com/pingcap/tiflow/pkg/sink/kafka"
    35  	"go.uber.org/atomic"
    36  	"go.uber.org/zap"
    37  )
    38  
    39  // Assert EventSink[E event.TableEvent] implementation
    40  var _ dmlsink.EventSink[*model.SingleTableTxn] = (*dmlSink)(nil)
    41  
    42  // dmlSink is the mq sink.
    43  // It will send the events to the MQ system.
    44  type dmlSink struct {
    45  	// id indicates this sink belongs to which processor(changefeed).
    46  	id model.ChangeFeedID
    47  	// protocol indicates the protocol used by this sink.
    48  	protocol config.Protocol
    49  
    50  	alive struct {
    51  		sync.RWMutex
    52  
    53  		transformer transformer.Transformer
    54  		// eventRouter used to route events to the right topic and partition.
    55  		eventRouter *dispatcher.EventRouter
    56  		// topicManager used to manage topics.
    57  		// It is also responsible for creating topics.
    58  		topicManager manager.TopicManager
    59  		worker       *worker
    60  		isDead       bool
    61  	}
    62  
    63  	// adminClient is used to query kafka cluster information, it's shared among
    64  	// multiple place, it's sink's responsibility to close it.
    65  	adminClient kafka.ClusterAdminClient
    66  
    67  	ctx    context.Context
    68  	cancel context.CancelCauseFunc
    69  
    70  	wg   sync.WaitGroup
    71  	dead chan struct{}
    72  
    73  	scheme string
    74  }
    75  
    76  func newDMLSink(
    77  	ctx context.Context,
    78  	changefeedID model.ChangeFeedID,
    79  	producer dmlproducer.DMLProducer,
    80  	adminClient kafka.ClusterAdminClient,
    81  	topicManager manager.TopicManager,
    82  	eventRouter *dispatcher.EventRouter,
    83  	transformer transformer.Transformer,
    84  	encoderGroup codec.EncoderGroup,
    85  	protocol config.Protocol,
    86  	scheme string,
    87  	errCh chan error,
    88  ) *dmlSink {
    89  	ctx, cancel := context.WithCancelCause(ctx)
    90  	statistics := metrics.NewStatistics(ctx, changefeedID, sink.RowSink)
    91  	worker := newWorker(changefeedID, protocol, producer, encoderGroup, statistics)
    92  
    93  	s := &dmlSink{
    94  		id:          changefeedID,
    95  		protocol:    protocol,
    96  		adminClient: adminClient,
    97  		ctx:         ctx,
    98  		cancel:      cancel,
    99  		dead:        make(chan struct{}),
   100  		scheme:      scheme,
   101  	}
   102  	s.alive.transformer = transformer
   103  	s.alive.eventRouter = eventRouter
   104  	s.alive.topicManager = topicManager
   105  	s.alive.worker = worker
   106  
   107  	// Spawn a goroutine to send messages by the worker.
   108  	s.wg.Add(1)
   109  	go func() {
   110  		defer s.wg.Done()
   111  		err := s.alive.worker.run(ctx)
   112  
   113  		s.alive.Lock()
   114  		s.alive.isDead = true
   115  		s.alive.worker.close()
   116  		s.alive.Unlock()
   117  		close(s.dead)
   118  
   119  		if err != nil {
   120  			if errors.Cause(err) == context.Canceled {
   121  				err = context.Cause(ctx)
   122  			}
   123  			select {
   124  			case errCh <- err:
   125  				log.Warn("mq dml sink meet error",
   126  					zap.String("namespace", s.id.Namespace),
   127  					zap.String("changefeed", s.id.ID),
   128  					zap.Error(err))
   129  			default:
   130  				log.Info("mq dml sink meet error, ignored",
   131  					zap.String("namespace", s.id.Namespace),
   132  					zap.String("changefeed", s.id.ID),
   133  					zap.Error(err))
   134  			}
   135  		}
   136  	}()
   137  
   138  	return s
   139  }
   140  
   141  // WriteEvents writes events to the sink.
   142  // This is an asynchronously and thread-safe method.
   143  func (s *dmlSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTableTxn]) error {
   144  	s.alive.RLock()
   145  	defer s.alive.RUnlock()
   146  	if s.alive.isDead {
   147  		return errors.Trace(errors.New("dead dmlSink"))
   148  	}
   149  	// Because we split txn to rows when sending to the MQ.
   150  	// So we need to convert the txn level callback to row level callback.
   151  	toRowCallback := func(txnCallback func(), totalCount uint64) func() {
   152  		var calledCount atomic.Uint64
   153  		// The callback of the last row will trigger the callback of the txn.
   154  		return func() {
   155  			if calledCount.Inc() == totalCount {
   156  				txnCallback()
   157  			}
   158  		}
   159  	}
   160  
   161  	for _, txn := range txns {
   162  		if txn.GetTableSinkState() != state.TableSinkSinking {
   163  			// The table where the event comes from is in stopping, so it's safe
   164  			// to drop the event directly.
   165  			txn.Callback()
   166  			continue
   167  		}
   168  		rowCallback := toRowCallback(txn.Callback, uint64(len(txn.Event.Rows)))
   169  		for _, row := range txn.Event.Rows {
   170  			topic := s.alive.eventRouter.GetTopicForRowChange(row)
   171  			partitionNum, err := s.alive.topicManager.GetPartitionNum(s.ctx, topic)
   172  			failpoint.Inject("MQSinkGetPartitionError", func() {
   173  				log.Info("failpoint MQSinkGetPartitionError injected", zap.String("changefeedID", s.id.ID))
   174  				err = errors.New("MQSinkGetPartitionError")
   175  			})
   176  			if err != nil {
   177  				s.cancel(err)
   178  				return errors.Trace(err)
   179  			}
   180  
   181  			err = s.alive.transformer.Apply(row)
   182  			if err != nil {
   183  				s.cancel(err)
   184  				return errors.Trace(err)
   185  			}
   186  			// Note: Calculate the partition index after the transformer is applied.
   187  			// Because the transformer may change the row of the event.
   188  			index, key, err := s.alive.eventRouter.GetPartitionForRowChange(row, partitionNum)
   189  			if err != nil {
   190  				s.cancel(err)
   191  				return errors.Trace(err)
   192  			}
   193  
   194  			// This never be blocked because this is an unbounded channel.
   195  			// We already limit the memory usage by MemoryQuota at SinkManager level.
   196  			// So it is safe to send the event to a unbounded channel here.
   197  			s.alive.worker.msgChan.In() <- mqEvent{
   198  				key: model.TopicPartitionKey{
   199  					Topic:          topic,
   200  					Partition:      index,
   201  					PartitionKey:   key,
   202  					TotalPartition: partitionNum,
   203  				},
   204  				rowEvent: &dmlsink.RowChangeCallbackableEvent{
   205  					Event:     row,
   206  					Callback:  rowCallback,
   207  					SinkState: txn.SinkState,
   208  				},
   209  			}
   210  		}
   211  	}
   212  	return nil
   213  }
   214  
   215  // Close closes the sink.
   216  func (s *dmlSink) Close() {
   217  	if s.cancel != nil {
   218  		s.cancel(nil)
   219  	}
   220  	s.wg.Wait()
   221  
   222  	s.alive.RLock()
   223  	if s.alive.topicManager != nil {
   224  		s.alive.topicManager.Close()
   225  	}
   226  	s.alive.RUnlock()
   227  
   228  	if s.adminClient != nil {
   229  		s.adminClient.Close()
   230  	}
   231  }
   232  
   233  // Dead checks whether it's dead or not.
   234  func (s *dmlSink) Dead() <-chan struct{} {
   235  	return s.dead
   236  }
   237  
   238  // Scheme returns the scheme of this sink.
   239  func (s *dmlSink) Scheme() string {
   240  	return s.scheme
   241  }