github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/producer/kafka/kafka.go (about)

     1  // Copyright 2020 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 kafka
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"regexp"
    20  	"strings"
    21  	"sync"
    22  	"sync/atomic"
    23  	"time"
    24  
    25  	"github.com/Shopify/sarama"
    26  	"github.com/pingcap/errors"
    27  	"github.com/pingcap/failpoint"
    28  	"github.com/pingcap/log"
    29  	"github.com/pingcap/ticdc/cdc/sink/codec"
    30  	cerror "github.com/pingcap/ticdc/pkg/errors"
    31  	"github.com/pingcap/ticdc/pkg/notify"
    32  	"github.com/pingcap/ticdc/pkg/security"
    33  	"github.com/pingcap/ticdc/pkg/util"
    34  	"go.uber.org/zap"
    35  )
    36  
    37  // Config stores the Kafka configuration
    38  type Config struct {
    39  	PartitionNum      int32
    40  	ReplicationFactor int16
    41  
    42  	Version         string
    43  	MaxMessageBytes int
    44  	Compression     string
    45  	ClientID        string
    46  	Credential      *security.Credential
    47  	SaslScram       *security.SaslScram
    48  	// control whether to create topic and verify partition number
    49  	TopicPreProcess bool
    50  }
    51  
    52  // NewKafkaConfig returns a default Kafka configuration
    53  func NewKafkaConfig() Config {
    54  	return Config{
    55  		Version:           "2.4.0",
    56  		MaxMessageBytes:   512 * 1024 * 1024, // 512M
    57  		ReplicationFactor: 1,
    58  		Compression:       "none",
    59  		Credential:        &security.Credential{},
    60  		SaslScram:         &security.SaslScram{},
    61  		TopicPreProcess:   true,
    62  	}
    63  }
    64  
    65  type kafkaSaramaProducer struct {
    66  	// clientLock is used to protect concurrent access of asyncClient and syncClient.
    67  	// Since we don't close these two clients (which have a input chan) from the
    68  	// sender routine, data race or send on closed chan could happen.
    69  	clientLock   sync.RWMutex
    70  	asyncClient  sarama.AsyncProducer
    71  	syncClient   sarama.SyncProducer
    72  	topic        string
    73  	partitionNum int32
    74  
    75  	partitionOffset []struct {
    76  		flushed uint64
    77  		sent    uint64
    78  	}
    79  	flushedNotifier *notify.Notifier
    80  	flushedReceiver *notify.Receiver
    81  
    82  	failpointCh chan error
    83  
    84  	closeCh chan struct{}
    85  	closed  int32
    86  }
    87  
    88  func (k *kafkaSaramaProducer) SendMessage(ctx context.Context, message *codec.MQMessage, partition int32) error {
    89  	k.clientLock.RLock()
    90  	defer k.clientLock.RUnlock()
    91  	msg := &sarama.ProducerMessage{
    92  		Topic:     k.topic,
    93  		Key:       sarama.ByteEncoder(message.Key),
    94  		Value:     sarama.ByteEncoder(message.Value),
    95  		Partition: partition,
    96  	}
    97  	msg.Metadata = atomic.AddUint64(&k.partitionOffset[partition].sent, 1)
    98  
    99  	failpoint.Inject("KafkaSinkAsyncSendError", func() {
   100  		// simulate sending message to input channel successfully but flushing
   101  		// message to Kafka meets error
   102  		log.Info("failpoint error injected")
   103  		k.failpointCh <- errors.New("kafka sink injected error")
   104  		failpoint.Return(nil)
   105  	})
   106  
   107  	failpoint.Inject("SinkFlushDMLPanic", func() {
   108  		time.Sleep(time.Second)
   109  		log.Panic("SinkFlushDMLPanic")
   110  	})
   111  
   112  	select {
   113  	case <-ctx.Done():
   114  		return ctx.Err()
   115  	case <-k.closeCh:
   116  		return nil
   117  	default:
   118  		k.asyncClient.Input() <- msg
   119  	}
   120  	return nil
   121  }
   122  
   123  func (k *kafkaSaramaProducer) SyncBroadcastMessage(ctx context.Context, message *codec.MQMessage) error {
   124  	k.clientLock.RLock()
   125  	defer k.clientLock.RUnlock()
   126  	msgs := make([]*sarama.ProducerMessage, k.partitionNum)
   127  	for i := 0; i < int(k.partitionNum); i++ {
   128  		msgs[i] = &sarama.ProducerMessage{
   129  			Topic:     k.topic,
   130  			Key:       sarama.ByteEncoder(message.Key),
   131  			Value:     sarama.ByteEncoder(message.Value),
   132  			Partition: int32(i),
   133  		}
   134  	}
   135  	select {
   136  	case <-ctx.Done():
   137  		return ctx.Err()
   138  	case <-k.closeCh:
   139  		return nil
   140  	default:
   141  		err := k.syncClient.SendMessages(msgs)
   142  		return cerror.WrapError(cerror.ErrKafkaSendMessage, err)
   143  	}
   144  }
   145  
   146  func (k *kafkaSaramaProducer) Flush(ctx context.Context) error {
   147  	targetOffsets := make([]uint64, k.partitionNum)
   148  	for i := 0; i < len(k.partitionOffset); i++ {
   149  		targetOffsets[i] = atomic.LoadUint64(&k.partitionOffset[i].sent)
   150  	}
   151  
   152  	noEventsToFLush := true
   153  	for i, target := range targetOffsets {
   154  		if target > atomic.LoadUint64(&k.partitionOffset[i].flushed) {
   155  			noEventsToFLush = false
   156  			break
   157  		}
   158  	}
   159  	if noEventsToFLush {
   160  		// no events to flush
   161  		return nil
   162  	}
   163  
   164  	// checkAllPartitionFlushed checks whether data in each partition is flushed
   165  	checkAllPartitionFlushed := func() bool {
   166  		for i, target := range targetOffsets {
   167  			if target > atomic.LoadUint64(&k.partitionOffset[i].flushed) {
   168  				return false
   169  			}
   170  		}
   171  		return true
   172  	}
   173  
   174  flushLoop:
   175  	for {
   176  		select {
   177  		case <-ctx.Done():
   178  			return ctx.Err()
   179  		case <-k.closeCh:
   180  			if checkAllPartitionFlushed() {
   181  				return nil
   182  			}
   183  			return cerror.ErrKafkaFlushUnfinished.GenWithStackByArgs()
   184  		case <-k.flushedReceiver.C:
   185  			if !checkAllPartitionFlushed() {
   186  				continue flushLoop
   187  			}
   188  			return nil
   189  		}
   190  	}
   191  }
   192  
   193  func (k *kafkaSaramaProducer) GetPartitionNum() int32 {
   194  	return k.partitionNum
   195  }
   196  
   197  // stop closes the closeCh to signal other routines to exit
   198  func (k *kafkaSaramaProducer) stop() {
   199  	k.clientLock.Lock()
   200  	defer k.clientLock.Unlock()
   201  	select {
   202  	case <-k.closeCh:
   203  		return
   204  	default:
   205  		close(k.closeCh)
   206  	}
   207  }
   208  
   209  // Close implements the Producer interface
   210  func (k *kafkaSaramaProducer) Close() error {
   211  	k.stop()
   212  	k.clientLock.Lock()
   213  	defer k.clientLock.Unlock()
   214  	// close sarama client multiple times will cause panic
   215  	if atomic.LoadInt32(&k.closed) == 1 {
   216  		return nil
   217  	}
   218  	// In fact close sarama sync client doesn't return any error.
   219  	// But close async client returns error if error channel is not empty, we
   220  	// don't populate this error to the upper caller, just add a log here.
   221  	err1 := k.syncClient.Close()
   222  	err2 := k.asyncClient.Close()
   223  	if err1 != nil {
   224  		log.Error("close sync client with error", zap.Error(err1))
   225  	}
   226  	if err2 != nil {
   227  		log.Error("close async client with error", zap.Error(err2))
   228  	}
   229  	atomic.StoreInt32(&k.closed, 1)
   230  	return nil
   231  }
   232  
   233  func (k *kafkaSaramaProducer) run(ctx context.Context) error {
   234  	defer func() {
   235  		k.flushedReceiver.Stop()
   236  		k.stop()
   237  	}()
   238  	for {
   239  		select {
   240  		case <-ctx.Done():
   241  			return ctx.Err()
   242  		case <-k.closeCh:
   243  			return nil
   244  		case err := <-k.failpointCh:
   245  			log.Warn("receive from failpoint chan", zap.Error(err))
   246  			return err
   247  		case msg := <-k.asyncClient.Successes():
   248  			if msg == nil || msg.Metadata == nil {
   249  				continue
   250  			}
   251  			flushedOffset := msg.Metadata.(uint64)
   252  			atomic.StoreUint64(&k.partitionOffset[msg.Partition].flushed, flushedOffset)
   253  			k.flushedNotifier.Notify()
   254  		case err := <-k.asyncClient.Errors():
   255  			// We should not wrap a nil pointer if the pointer is of a subtype of `error`
   256  			// because Go would store the type info and the resulted `error` variable would not be nil,
   257  			// which will cause the pkg/error library to malfunction.
   258  			if err == nil {
   259  				return nil
   260  			}
   261  			return cerror.WrapError(cerror.ErrKafkaAsyncSendMessage, err)
   262  		}
   263  	}
   264  }
   265  
   266  // kafkaTopicPreProcess gets partition number from existing topic, if topic doesn't
   267  // exit, creates it automatically.
   268  func kafkaTopicPreProcess(topic, address string, config Config, cfg *sarama.Config) (int32, error) {
   269  	admin, err := sarama.NewClusterAdmin(strings.Split(address, ","), cfg)
   270  	if err != nil {
   271  		return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err)
   272  	}
   273  	defer func() {
   274  		err := admin.Close()
   275  		if err != nil {
   276  			log.Warn("close admin client failed", zap.Error(err))
   277  		}
   278  	}()
   279  	topics, err := admin.ListTopics()
   280  	if err != nil {
   281  		return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err)
   282  	}
   283  	partitionNum := config.PartitionNum
   284  	topicDetail, exist := topics[topic]
   285  	if exist {
   286  		log.Info("get partition number of topic", zap.String("topic", topic), zap.Int32("partition_num", topicDetail.NumPartitions))
   287  		if partitionNum == 0 {
   288  			partitionNum = topicDetail.NumPartitions
   289  		} else if partitionNum < topicDetail.NumPartitions {
   290  			log.Warn("partition number assigned in sink-uri is less than that of topic", zap.Int32("topic partition num", topicDetail.NumPartitions))
   291  		} else if partitionNum > topicDetail.NumPartitions {
   292  			return 0, cerror.ErrKafkaInvalidPartitionNum.GenWithStack(
   293  				"partition number(%d) assigned in sink-uri is more than that of topic(%d)", partitionNum, topicDetail.NumPartitions)
   294  		}
   295  	} else {
   296  		if partitionNum == 0 {
   297  			partitionNum = 4
   298  			log.Warn("topic not found and partition number is not specified, using default partition number", zap.String("topic", topic), zap.Int32("partition_num", partitionNum))
   299  		}
   300  		log.Info("create a topic", zap.String("topic", topic),
   301  			zap.Int32("partition_num", partitionNum),
   302  			zap.Int16("replication_factor", config.ReplicationFactor))
   303  		err := admin.CreateTopic(topic, &sarama.TopicDetail{
   304  			NumPartitions:     partitionNum,
   305  			ReplicationFactor: config.ReplicationFactor,
   306  		}, false)
   307  		// TODO idenfity the cause of "Topic with this name already exists"
   308  		if err != nil && !strings.Contains(err.Error(), "already exists") {
   309  			return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err)
   310  		}
   311  	}
   312  
   313  	return partitionNum, nil
   314  }
   315  
   316  var newSaramaConfigImpl = newSaramaConfig
   317  
   318  // NewKafkaSaramaProducer creates a kafka sarama producer
   319  func NewKafkaSaramaProducer(ctx context.Context, address string, topic string, config Config, errCh chan error) (*kafkaSaramaProducer, error) {
   320  	log.Info("Starting kafka sarama producer ...", zap.Reflect("config", config))
   321  	cfg, err := newSaramaConfigImpl(ctx, config)
   322  	if err != nil {
   323  		return nil, err
   324  	}
   325  	if config.PartitionNum < 0 {
   326  		return nil, cerror.ErrKafkaInvalidPartitionNum.GenWithStackByArgs(config.PartitionNum)
   327  	}
   328  	asyncClient, err := sarama.NewAsyncProducer(strings.Split(address, ","), cfg)
   329  	if err != nil {
   330  		return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err)
   331  	}
   332  	syncClient, err := sarama.NewSyncProducer(strings.Split(address, ","), cfg)
   333  	if err != nil {
   334  		return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err)
   335  	}
   336  
   337  	partitionNum := config.PartitionNum
   338  	if config.TopicPreProcess {
   339  		partitionNum, err = kafkaTopicPreProcess(topic, address, config, cfg)
   340  		if err != nil {
   341  			return nil, err
   342  		}
   343  	}
   344  
   345  	notifier := new(notify.Notifier)
   346  	flushedReceiver, err := notifier.NewReceiver(50 * time.Millisecond)
   347  	if err != nil {
   348  		return nil, err
   349  	}
   350  	k := &kafkaSaramaProducer{
   351  		asyncClient:  asyncClient,
   352  		syncClient:   syncClient,
   353  		topic:        topic,
   354  		partitionNum: partitionNum,
   355  		partitionOffset: make([]struct {
   356  			flushed uint64
   357  			sent    uint64
   358  		}, partitionNum),
   359  		flushedNotifier: notifier,
   360  		flushedReceiver: flushedReceiver,
   361  		closeCh:         make(chan struct{}),
   362  		failpointCh:     make(chan error, 1),
   363  	}
   364  	go func() {
   365  		if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled {
   366  			select {
   367  			case <-ctx.Done():
   368  				return
   369  			case errCh <- err:
   370  			default:
   371  				log.Error("error channel is full", zap.Error(err))
   372  			}
   373  		}
   374  	}()
   375  	return k, nil
   376  }
   377  
   378  func init() {
   379  	sarama.MaxRequestSize = 1024 * 1024 * 1024 // 1GB
   380  }
   381  
   382  var (
   383  	validClientID     *regexp.Regexp = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`)
   384  	commonInvalidChar *regexp.Regexp = regexp.MustCompile(`[\?:,"]`)
   385  )
   386  
   387  func kafkaClientID(role, captureAddr, changefeedID, configuredClientID string) (clientID string, err error) {
   388  	if configuredClientID != "" {
   389  		clientID = configuredClientID
   390  	} else {
   391  		clientID = fmt.Sprintf("TiCDC_sarama_producer_%s_%s_%s", role, captureAddr, changefeedID)
   392  		clientID = commonInvalidChar.ReplaceAllString(clientID, "_")
   393  	}
   394  	if !validClientID.MatchString(clientID) {
   395  		return "", cerror.ErrKafkaInvalidClientID.GenWithStackByArgs(clientID)
   396  	}
   397  	return
   398  }
   399  
   400  // NewSaramaConfig return the default config and set the according version and metrics
   401  func newSaramaConfig(ctx context.Context, c Config) (*sarama.Config, error) {
   402  	config := sarama.NewConfig()
   403  
   404  	version, err := sarama.ParseKafkaVersion(c.Version)
   405  	if err != nil {
   406  		return nil, cerror.WrapError(cerror.ErrKafkaInvalidVersion, err)
   407  	}
   408  	var role string
   409  	if util.IsOwnerFromCtx(ctx) {
   410  		role = "owner"
   411  	} else {
   412  		role = "processor"
   413  	}
   414  	captureAddr := util.CaptureAddrFromCtx(ctx)
   415  	changefeedID := util.ChangefeedIDFromCtx(ctx)
   416  
   417  	config.ClientID, err = kafkaClientID(role, captureAddr, changefeedID, c.ClientID)
   418  	if err != nil {
   419  		return nil, errors.Trace(err)
   420  	}
   421  	config.Version = version
   422  	// See: https://kafka.apache.org/documentation/#replication
   423  	// When one of the brokers in a Kafka cluster is down, the partition leaders in this broker is broken, Kafka will election a new partition leader and replication logs, this process will last from a few seconds to a few minutes. Kafka cluster will not provide a writing service in this process.
   424  	// Time out in one minute(120 * 500ms).
   425  	config.Metadata.Retry.Max = 120
   426  	config.Metadata.Retry.Backoff = 500 * time.Millisecond
   427  
   428  	config.Producer.Partitioner = sarama.NewManualPartitioner
   429  	config.Producer.MaxMessageBytes = c.MaxMessageBytes
   430  	config.Producer.Return.Successes = true
   431  	config.Producer.Return.Errors = true
   432  	config.Producer.RequiredAcks = sarama.WaitForAll
   433  
   434  	switch strings.ToLower(strings.TrimSpace(c.Compression)) {
   435  	case "none":
   436  		config.Producer.Compression = sarama.CompressionNone
   437  	case "gzip":
   438  		config.Producer.Compression = sarama.CompressionGZIP
   439  	case "snappy":
   440  		config.Producer.Compression = sarama.CompressionSnappy
   441  	case "lz4":
   442  		config.Producer.Compression = sarama.CompressionLZ4
   443  	case "zstd":
   444  		config.Producer.Compression = sarama.CompressionZSTD
   445  	default:
   446  		log.Warn("Unsupported compression algorithm", zap.String("compression", c.Compression))
   447  		config.Producer.Compression = sarama.CompressionNone
   448  	}
   449  
   450  	// Time out in five minutes(600 * 500ms).
   451  	config.Producer.Retry.Max = 600
   452  	config.Producer.Retry.Backoff = 500 * time.Millisecond
   453  
   454  	// Time out in one minute(120 * 500ms).
   455  	config.Admin.Retry.Max = 120
   456  	config.Admin.Retry.Backoff = 500 * time.Millisecond
   457  	config.Admin.Timeout = 20 * time.Second
   458  
   459  	if c.Credential != nil && len(c.Credential.CAPath) != 0 {
   460  		config.Net.TLS.Enable = true
   461  		config.Net.TLS.Config, err = c.Credential.ToTLSConfig()
   462  		if err != nil {
   463  			return nil, errors.Trace(err)
   464  		}
   465  	}
   466  
   467  	if c.SaslScram != nil && len(c.SaslScram.SaslUser) != 0 {
   468  		config.Net.SASL.Enable = true
   469  		config.Net.SASL.User = c.SaslScram.SaslUser
   470  		config.Net.SASL.Password = c.SaslScram.SaslPassword
   471  		config.Net.SASL.Mechanism = sarama.SASLMechanism(c.SaslScram.SaslMechanism)
   472  		if strings.EqualFold(c.SaslScram.SaslMechanism, "SCRAM-SHA-256") {
   473  			config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &security.XDGSCRAMClient{HashGeneratorFcn: security.SHA256} }
   474  		} else if strings.EqualFold(c.SaslScram.SaslMechanism, "SCRAM-SHA-512") {
   475  			config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { return &security.XDGSCRAMClient{HashGeneratorFcn: security.SHA512} }
   476  		} else {
   477  			return nil, errors.New("Unsupported sasl-mechanism, should be SCRAM-SHA-256 or SCRAM-SHA-512")
   478  		}
   479  	}
   480  
   481  	return config, err
   482  }