github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/pulsar_dml_sink.go (about)

     1  // Copyright 2023 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  	"net/url"
    19  	"time"
    20  
    21  	"github.com/pingcap/errors"
    22  	"github.com/pingcap/log"
    23  	"github.com/pingcap/tiflow/cdc/model"
    24  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher"
    25  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dmlproducer"
    26  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/manager"
    27  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/transformer/columnselector"
    28  	"github.com/pingcap/tiflow/cdc/sink/util"
    29  	"github.com/pingcap/tiflow/pkg/config"
    30  	cerror "github.com/pingcap/tiflow/pkg/errors"
    31  	"github.com/pingcap/tiflow/pkg/sink"
    32  	"github.com/pingcap/tiflow/pkg/sink/codec"
    33  	"github.com/pingcap/tiflow/pkg/sink/codec/builder"
    34  	pulsarConfig "github.com/pingcap/tiflow/pkg/sink/pulsar"
    35  	tiflowutil "github.com/pingcap/tiflow/pkg/util"
    36  	"go.uber.org/zap"
    37  )
    38  
    39  // NewPulsarDMLSink will verify the config and create a PulsarSink.
    40  func NewPulsarDMLSink(
    41  	ctx context.Context,
    42  	changefeedID model.ChangeFeedID,
    43  	sinkURI *url.URL,
    44  	replicaConfig *config.ReplicaConfig,
    45  	errCh chan error,
    46  	pulsarTopicManagerCreator manager.PulsarTopicManager,
    47  	clientCreator pulsarConfig.FactoryCreator,
    48  	producerCreator dmlproducer.PulsarFactory,
    49  ) (_ *dmlSink, err error) {
    50  	log.Info("Starting pulsar DML producer ...",
    51  		zap.String("namespace", changefeedID.Namespace),
    52  		zap.String("changefeed", changefeedID.ID))
    53  
    54  	defaultTopic, err := util.GetTopic(sinkURI)
    55  	if err != nil {
    56  		return nil, errors.Trace(err)
    57  	}
    58  
    59  	protocol, err := util.GetProtocol(tiflowutil.GetOrZero(replicaConfig.Sink.Protocol))
    60  	if err != nil {
    61  		return nil, errors.Trace(err)
    62  	}
    63  	if !util.IsPulsarSupportedProtocols(protocol) {
    64  		return nil, cerror.ErrSinkURIInvalid.
    65  			GenWithStackByArgs("unsupported protocol, " +
    66  				"pulsar sink currently only support these protocols: [canal-json, canal, maxwell]")
    67  	}
    68  
    69  	pConfig, err := pulsarConfig.NewPulsarConfig(sinkURI, replicaConfig.Sink.PulsarConfig)
    70  	if err != nil {
    71  		return nil, errors.Trace(err)
    72  	}
    73  
    74  	client, err := clientCreator(pConfig, changefeedID, replicaConfig.Sink)
    75  	if err != nil {
    76  		log.Error("DML sink producer client create fail", zap.Error(err))
    77  		return nil, cerror.WrapError(cerror.ErrPulsarNewClient, err)
    78  	}
    79  
    80  	failpointCh := make(chan error, 1)
    81  	log.Info("Try to create a DML sink producer", zap.String("changefeed", changefeedID.String()))
    82  	start := time.Now()
    83  	p, err := producerCreator(ctx, changefeedID, client, replicaConfig.Sink, errCh, failpointCh)
    84  	log.Info("DML sink producer created",
    85  		zap.String("changefeed", changefeedID.String()),
    86  		zap.Duration("duration", time.Since(start)))
    87  	if err != nil {
    88  		defer func() {
    89  			if p != nil {
    90  				p.Close()
    91  			}
    92  		}()
    93  		return nil, cerror.WrapError(cerror.ErrPulsarNewProducer, err)
    94  	}
    95  
    96  	scheme := sink.GetScheme(sinkURI)
    97  	// The topicManager is not actually used in pulsar , it is only used to create dmlSink.
    98  	// TODO: Find a way to remove it in newDMLSink.
    99  	topicManager, err := pulsarTopicManagerCreator(pConfig, client)
   100  	if err != nil {
   101  		return nil, errors.Trace(err)
   102  	}
   103  	eventRouter, err := dispatcher.NewEventRouter(replicaConfig, protocol, defaultTopic, scheme)
   104  	if err != nil {
   105  		return nil, errors.Trace(err)
   106  	}
   107  
   108  	trans, err := columnselector.New(replicaConfig)
   109  	if err != nil {
   110  		return nil, errors.Trace(err)
   111  	}
   112  
   113  	encoderConfig, err := util.GetEncoderConfig(changefeedID, sinkURI, protocol, replicaConfig,
   114  		config.DefaultMaxMessageBytes)
   115  	if err != nil {
   116  		return nil, errors.Trace(err)
   117  	}
   118  
   119  	encoderBuilder, err := builder.NewRowEventEncoderBuilder(ctx, encoderConfig)
   120  	if err != nil {
   121  		return nil, cerror.WrapError(cerror.ErrPulsarInvalidConfig, err)
   122  	}
   123  
   124  	encoderGroup := codec.NewEncoderGroup(replicaConfig.Sink, encoderBuilder, changefeedID)
   125  
   126  	s := newDMLSink(ctx, changefeedID, p, nil, topicManager,
   127  		eventRouter, trans, encoderGroup, protocol, scheme, errCh)
   128  
   129  	return s, nil
   130  }