github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/kafka_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  	"net/url"
    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/mq/dispatcher"
    24  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dmlproducer"
    25  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/transformer/columnselector"
    26  	"github.com/pingcap/tiflow/cdc/sink/util"
    27  	"github.com/pingcap/tiflow/pkg/config"
    28  	cerror "github.com/pingcap/tiflow/pkg/errors"
    29  	"github.com/pingcap/tiflow/pkg/sink"
    30  	"github.com/pingcap/tiflow/pkg/sink/codec"
    31  	"github.com/pingcap/tiflow/pkg/sink/codec/builder"
    32  	"github.com/pingcap/tiflow/pkg/sink/kafka"
    33  	tiflowutil "github.com/pingcap/tiflow/pkg/util"
    34  	"go.uber.org/zap"
    35  )
    36  
    37  // NewKafkaDMLSink will verify the config and create a KafkaSink.
    38  func NewKafkaDMLSink(
    39  	ctx context.Context,
    40  	changefeedID model.ChangeFeedID,
    41  	sinkURI *url.URL,
    42  	replicaConfig *config.ReplicaConfig,
    43  	errCh chan error,
    44  	factoryCreator kafka.FactoryCreator,
    45  	producerCreator dmlproducer.Factory,
    46  ) (_ *dmlSink, err error) {
    47  	topic, err := util.GetTopic(sinkURI)
    48  	if err != nil {
    49  		return nil, errors.Trace(err)
    50  	}
    51  
    52  	options := kafka.NewOptions()
    53  	if err := options.Apply(changefeedID, sinkURI, replicaConfig); err != nil {
    54  		return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err)
    55  	}
    56  
    57  	factory, err := factoryCreator(options, changefeedID)
    58  	if err != nil {
    59  		return nil, cerror.WrapError(cerror.ErrKafkaNewProducer, err)
    60  	}
    61  
    62  	adminClient, err := factory.AdminClient(ctx)
    63  	if err != nil {
    64  		return nil, cerror.WrapError(cerror.ErrKafkaNewProducer, err)
    65  	}
    66  	// We must close adminClient when this func return cause by an error
    67  	// otherwise the adminClient will never be closed and lead to a goroutine leak.
    68  	defer func() {
    69  		if err != nil && adminClient != nil {
    70  			adminClient.Close()
    71  		}
    72  	}()
    73  
    74  	// adjust the option configuration before creating the kafka client
    75  	if err = kafka.AdjustOptions(ctx, adminClient, options, topic); err != nil {
    76  		return nil, cerror.WrapError(cerror.ErrKafkaNewProducer, err)
    77  	}
    78  
    79  	protocol, err := util.GetProtocol(tiflowutil.GetOrZero(replicaConfig.Sink.Protocol))
    80  	if err != nil {
    81  		return nil, errors.Trace(err)
    82  	}
    83  
    84  	topicManager, err := util.GetTopicManagerAndTryCreateTopic(
    85  		ctx,
    86  		changefeedID,
    87  		topic,
    88  		options.DeriveTopicConfig(),
    89  		adminClient,
    90  	)
    91  	if err != nil {
    92  		return nil, errors.Trace(err)
    93  	}
    94  
    95  	scheme := sink.GetScheme(sinkURI)
    96  	eventRouter, err := dispatcher.NewEventRouter(replicaConfig, protocol, topic, scheme)
    97  	if err != nil {
    98  		return nil, errors.Trace(err)
    99  	}
   100  
   101  	trans, err := columnselector.New(replicaConfig)
   102  	if err != nil {
   103  		return nil, errors.Trace(err)
   104  	}
   105  
   106  	encoderConfig, err := util.GetEncoderConfig(changefeedID, sinkURI, protocol, replicaConfig, options.MaxMessageBytes)
   107  	if err != nil {
   108  		return nil, errors.Trace(err)
   109  	}
   110  
   111  	encoderBuilder, err := builder.NewRowEventEncoderBuilder(ctx, encoderConfig)
   112  	if err != nil {
   113  		return nil, cerror.WrapError(cerror.ErrKafkaNewProducer, err)
   114  	}
   115  
   116  	failpointCh := make(chan error, 1)
   117  	asyncProducer, err := factory.AsyncProducer(ctx, failpointCh)
   118  	if err != nil {
   119  		return nil, cerror.WrapError(cerror.ErrKafkaNewProducer, err)
   120  	}
   121  
   122  	metricsCollector := factory.MetricsCollector(tiflowutil.RoleProcessor, adminClient)
   123  	dmlProducer := producerCreator(ctx, changefeedID, asyncProducer, metricsCollector, errCh, failpointCh)
   124  	encoderGroup := codec.NewEncoderGroup(replicaConfig.Sink, encoderBuilder, changefeedID)
   125  	s := newDMLSink(ctx, changefeedID, dmlProducer, adminClient, topicManager,
   126  		eventRouter, trans, encoderGroup, protocol, scheme, errCh)
   127  	log.Info("DML sink producer created",
   128  		zap.String("namespace", changefeedID.Namespace),
   129  		zap.String("changefeedID", changefeedID.ID))
   130  
   131  	return s, nil
   132  }