github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/cloudstorage/cloud_storage_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 cloudstorage
    15  
    16  import (
    17  	"context"
    18  	"math"
    19  	"net/url"
    20  	"strings"
    21  	"sync"
    22  	"sync/atomic"
    23  
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/log"
    26  	"github.com/pingcap/tiflow/cdc/model"
    27  	"github.com/pingcap/tiflow/cdc/sink/dmlsink"
    28  	"github.com/pingcap/tiflow/cdc/sink/metrics"
    29  	"github.com/pingcap/tiflow/cdc/sink/tablesink/state"
    30  	"github.com/pingcap/tiflow/cdc/sink/util"
    31  	"github.com/pingcap/tiflow/pkg/chann"
    32  	"github.com/pingcap/tiflow/pkg/config"
    33  	cerror "github.com/pingcap/tiflow/pkg/errors"
    34  	"github.com/pingcap/tiflow/pkg/pdutil"
    35  	"github.com/pingcap/tiflow/pkg/sink"
    36  	"github.com/pingcap/tiflow/pkg/sink/cloudstorage"
    37  	"github.com/pingcap/tiflow/pkg/sink/codec/builder"
    38  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    39  	putil "github.com/pingcap/tiflow/pkg/util"
    40  	"go.uber.org/zap"
    41  	"golang.org/x/sync/errgroup"
    42  )
    43  
    44  const (
    45  	defaultEncodingConcurrency = 8
    46  	defaultChannelSize         = 1024
    47  )
    48  
    49  // Assert EventSink[E event.TableEvent] implementation
    50  var _ dmlsink.EventSink[*model.SingleTableTxn] = (*DMLSink)(nil)
    51  
    52  // eventFragment is used to attach a sequence number to TxnCallbackableEvent.
    53  type eventFragment struct {
    54  	event          *dmlsink.TxnCallbackableEvent
    55  	versionedTable cloudstorage.VersionedTableName
    56  
    57  	// The sequence number is mainly useful for TxnCallbackableEvent defragmentation.
    58  	// e.g. TxnCallbackableEvent 1~5 are dispatched to a group of encoding workers, but the
    59  	// encoding completion time varies. Let's say the final completion sequence are 1,3,2,5,4,
    60  	// we can use the sequence numbers to do defragmentation so that the events can arrive
    61  	// at dmlWorker sequentially.
    62  	seqNumber uint64
    63  	// encodedMsgs denote the encoded messages after the event is handled in encodingWorker.
    64  	encodedMsgs []*common.Message
    65  }
    66  
    67  // DMLSink is the cloud storage sink.
    68  // It will send the events to cloud storage systems.
    69  // Messages are encoded in the specific protocol and then sent to the defragmenter.
    70  // The data flow is as follows: **data** -> encodingWorkers -> defragmenter -> dmlWorkers -> external storage
    71  // The defragmenter will defragment the out-of-order encoded messages and sends encoded
    72  // messages to individual dmlWorkers.
    73  // The dmlWorkers will write the encoded messages to external storage in parallel between different tables.
    74  type DMLSink struct {
    75  	changefeedID model.ChangeFeedID
    76  	scheme       string
    77  	// last sequence number
    78  	lastSeqNum uint64
    79  	// encodingWorkers defines a group of workers for encoding events.
    80  	encodingWorkers []*encodingWorker
    81  	// defragmenter is used to defragment the out-of-order encoded messages and
    82  	// sends encoded messages to individual dmlWorkers.
    83  	defragmenter *defragmenter
    84  	// workers defines a group of workers for writing events to external storage.
    85  	workers []*dmlWorker
    86  
    87  	alive struct {
    88  		sync.RWMutex
    89  		// msgCh is a channel to hold eventFragment.
    90  		// The caller of WriteEvents will write eventFragment to msgCh and
    91  		// the encodingWorkers will read eventFragment from msgCh to encode events.
    92  		msgCh  *chann.DrainableChann[eventFragment]
    93  		isDead bool
    94  	}
    95  
    96  	statistics *metrics.Statistics
    97  
    98  	cancel func()
    99  	wg     sync.WaitGroup
   100  	dead   chan struct{}
   101  }
   102  
   103  // NewDMLSink creates a cloud storage sink.
   104  func NewDMLSink(ctx context.Context,
   105  	changefeedID model.ChangeFeedID,
   106  	pdClock pdutil.Clock,
   107  	sinkURI *url.URL,
   108  	replicaConfig *config.ReplicaConfig,
   109  	errCh chan error,
   110  ) (*DMLSink, error) {
   111  	// create cloud storage config and then apply the params of sinkURI to it.
   112  	cfg := cloudstorage.NewConfig()
   113  	err := cfg.Apply(ctx, sinkURI, replicaConfig)
   114  	if err != nil {
   115  		return nil, err
   116  	}
   117  
   118  	// create an external storage.
   119  	storage, err := putil.GetExternalStorageFromURI(ctx, sinkURI.String())
   120  	if err != nil {
   121  		return nil, err
   122  	}
   123  
   124  	// fetch protocol from replicaConfig defined by changefeed config file.
   125  	protocol, err := util.GetProtocol(
   126  		putil.GetOrZero(replicaConfig.Sink.Protocol),
   127  	)
   128  	if err != nil {
   129  		return nil, errors.Trace(err)
   130  	}
   131  
   132  	// get cloud storage file extension according to the specific protocol.
   133  	ext := util.GetFileExtension(protocol)
   134  	// the last param maxMsgBytes is mainly to limit the size of a single message for
   135  	// batch protocols in mq scenario. In cloud storage sink, we just set it to max int.
   136  	encoderConfig, err := util.GetEncoderConfig(changefeedID, sinkURI, protocol, replicaConfig, math.MaxInt)
   137  	if err != nil {
   138  		return nil, errors.Trace(err)
   139  	}
   140  	encoderBuilder, err := builder.NewTxnEventEncoderBuilder(encoderConfig)
   141  	if err != nil {
   142  		return nil, cerror.WrapError(cerror.ErrStorageSinkInvalidConfig, err)
   143  	}
   144  
   145  	wgCtx, wgCancel := context.WithCancel(ctx)
   146  	s := &DMLSink{
   147  		changefeedID:    changefeedID,
   148  		scheme:          strings.ToLower(sinkURI.Scheme),
   149  		encodingWorkers: make([]*encodingWorker, defaultEncodingConcurrency),
   150  		workers:         make([]*dmlWorker, cfg.WorkerCount),
   151  		statistics:      metrics.NewStatistics(wgCtx, changefeedID, sink.TxnSink),
   152  		cancel:          wgCancel,
   153  		dead:            make(chan struct{}),
   154  	}
   155  	s.alive.msgCh = chann.NewAutoDrainChann[eventFragment]()
   156  
   157  	encodedOutCh := make(chan eventFragment, defaultChannelSize)
   158  	workerChannels := make([]*chann.DrainableChann[eventFragment], cfg.WorkerCount)
   159  
   160  	// create a group of encoding workers.
   161  	for i := 0; i < defaultEncodingConcurrency; i++ {
   162  		encoder := encoderBuilder.Build()
   163  		s.encodingWorkers[i] = newEncodingWorker(i, s.changefeedID, encoder, s.alive.msgCh.Out(), encodedOutCh)
   164  	}
   165  
   166  	// create a group of dml workers.
   167  	for i := 0; i < cfg.WorkerCount; i++ {
   168  		inputCh := chann.NewAutoDrainChann[eventFragment]()
   169  		s.workers[i] = newDMLWorker(i, s.changefeedID, storage, cfg, ext,
   170  			inputCh, pdClock, s.statistics)
   171  		workerChannels[i] = inputCh
   172  	}
   173  
   174  	// create defragmenter.
   175  	// The defragmenter is used to defragment the out-of-order encoded messages from encoding workers and
   176  	// sends encoded messages to related dmlWorkers in order. Messages of the same table will be sent to
   177  	// the same dmlWorker.
   178  	s.defragmenter = newDefragmenter(encodedOutCh, workerChannels)
   179  
   180  	s.wg.Add(1)
   181  	go func() {
   182  		defer s.wg.Done()
   183  		err := s.run(wgCtx)
   184  
   185  		s.alive.Lock()
   186  		s.alive.isDead = true
   187  		s.alive.msgCh.CloseAndDrain()
   188  		s.alive.Unlock()
   189  		close(s.dead)
   190  
   191  		if err != nil && errors.Cause(err) != context.Canceled {
   192  			select {
   193  			case <-wgCtx.Done():
   194  			case errCh <- err:
   195  			}
   196  		}
   197  	}()
   198  
   199  	return s, nil
   200  }
   201  
   202  func (s *DMLSink) run(ctx context.Context) error {
   203  	eg, ctx := errgroup.WithContext(ctx)
   204  
   205  	// run the encoding workers.
   206  	for i := 0; i < defaultEncodingConcurrency; i++ {
   207  		encodingWorker := s.encodingWorkers[i]
   208  		eg.Go(func() error {
   209  			return encodingWorker.run(ctx)
   210  		})
   211  	}
   212  
   213  	// run the defragmenter.
   214  	eg.Go(func() error {
   215  		return s.defragmenter.run(ctx)
   216  	})
   217  
   218  	// run dml workers.
   219  	for i := 0; i < len(s.workers); i++ {
   220  		worker := s.workers[i]
   221  		eg.Go(func() error {
   222  			return worker.run(ctx)
   223  		})
   224  	}
   225  
   226  	log.Info("dml worker started", zap.String("namespace", s.changefeedID.Namespace),
   227  		zap.String("changefeed", s.changefeedID.ID),
   228  		zap.Int("workerCount", len(s.workers)),
   229  		zap.Any("config", s.workers[0].config))
   230  
   231  	return eg.Wait()
   232  }
   233  
   234  // WriteEvents write events to cloud storage sink.
   235  func (s *DMLSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTableTxn]) error {
   236  	s.alive.RLock()
   237  	defer s.alive.RUnlock()
   238  	if s.alive.isDead {
   239  		return errors.Trace(errors.New("dead dmlSink"))
   240  	}
   241  
   242  	for _, txn := range txns {
   243  		if txn.GetTableSinkState() != state.TableSinkSinking {
   244  			// The table where the event comes from is in stopping, so it's safe
   245  			// to drop the event directly.
   246  			txn.Callback()
   247  			continue
   248  		}
   249  
   250  		tbl := cloudstorage.VersionedTableName{
   251  			TableNameWithPhysicTableID: model.TableName{
   252  				Schema:      txn.Event.TableInfo.GetSchemaName(),
   253  				Table:       txn.Event.TableInfo.GetTableName(),
   254  				TableID:     txn.Event.GetPhysicalTableID(),
   255  				IsPartition: txn.Event.TableInfo.IsPartitionTable(),
   256  			},
   257  			TableInfoVersion: txn.Event.TableInfoVersion,
   258  		}
   259  		seq := atomic.AddUint64(&s.lastSeqNum, 1)
   260  
   261  		s.statistics.ObserveRows(txn.Event.Rows...)
   262  		// emit a TxnCallbackableEvent encoupled with a sequence number starting from one.
   263  		s.alive.msgCh.In() <- eventFragment{
   264  			seqNumber:      seq,
   265  			versionedTable: tbl,
   266  			event:          txn,
   267  		}
   268  	}
   269  
   270  	return nil
   271  }
   272  
   273  // Close closes the cloud storage sink.
   274  func (s *DMLSink) Close() {
   275  	if s.cancel != nil {
   276  		s.cancel()
   277  	}
   278  	s.wg.Wait()
   279  
   280  	for _, encodingWorker := range s.encodingWorkers {
   281  		encodingWorker.close()
   282  	}
   283  
   284  	for _, worker := range s.workers {
   285  		worker.close()
   286  	}
   287  
   288  	if s.statistics != nil {
   289  		s.statistics.Close()
   290  	}
   291  }
   292  
   293  // Dead checks whether it's dead or not.
   294  func (s *DMLSink) Dead() <-chan struct{} {
   295  	return s.dead
   296  }
   297  
   298  // Scheme returns the sink scheme.
   299  func (s *DMLSink) Scheme() string {
   300  	return s.scheme
   301  }