github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/ddlsink/mq/ddlproducer/kafka_ddl_producer.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 ddlproducer
    15  
    16  import (
    17  	"context"
    18  	"sync"
    19  
    20  	"github.com/pingcap/errors"
    21  	"github.com/pingcap/log"
    22  	"github.com/pingcap/tiflow/cdc/model"
    23  	cerror "github.com/pingcap/tiflow/pkg/errors"
    24  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    25  	"github.com/pingcap/tiflow/pkg/sink/kafka"
    26  	"go.uber.org/zap"
    27  )
    28  
    29  // Assert DDLEventSink implementation
    30  var _ DDLProducer = (*kafkaDDLProducer)(nil)
    31  
    32  // kafkaDDLProducer is used to send messages to kafka synchronously.
    33  type kafkaDDLProducer struct {
    34  	// id indicates this sink belongs to which processor(changefeed).
    35  	id model.ChangeFeedID
    36  	// syncProducer is used to send messages to kafka synchronously.
    37  	syncProducer kafka.SyncProducer
    38  	// closedMu is used to protect `closed`.
    39  	// We need to ensure that closed producers are never written to.
    40  	closedMu sync.RWMutex
    41  	// closed is used to indicate whether the producer is closed.
    42  	// We also use it to guard against double closes.
    43  	closed bool
    44  }
    45  
    46  // NewKafkaDDLProducer creates a new kafka producer for replicating DDL.
    47  func NewKafkaDDLProducer(_ context.Context,
    48  	changefeedID model.ChangeFeedID,
    49  	syncProducer kafka.SyncProducer,
    50  ) DDLProducer {
    51  	return &kafkaDDLProducer{
    52  		id:           changefeedID,
    53  		syncProducer: syncProducer,
    54  		closed:       false,
    55  	}
    56  }
    57  
    58  func (k *kafkaDDLProducer) SyncBroadcastMessage(ctx context.Context, topic string,
    59  	totalPartitionsNum int32, message *common.Message,
    60  ) error {
    61  	k.closedMu.RLock()
    62  	defer k.closedMu.RUnlock()
    63  
    64  	if k.closed {
    65  		return cerror.ErrKafkaProducerClosed.GenWithStackByArgs()
    66  	}
    67  
    68  	select {
    69  	case <-ctx.Done():
    70  		return ctx.Err()
    71  	default:
    72  		err := k.syncProducer.SendMessages(ctx, topic,
    73  			totalPartitionsNum, message.Key, message.Value)
    74  		return cerror.WrapError(cerror.ErrKafkaSendMessage, err)
    75  	}
    76  }
    77  
    78  func (k *kafkaDDLProducer) SyncSendMessage(ctx context.Context, topic string,
    79  	partitionNum int32, message *common.Message,
    80  ) error {
    81  	k.closedMu.RLock()
    82  	defer k.closedMu.RUnlock()
    83  
    84  	if k.closed {
    85  		return cerror.ErrKafkaProducerClosed.GenWithStackByArgs()
    86  	}
    87  
    88  	select {
    89  	case <-ctx.Done():
    90  		return errors.Trace(ctx.Err())
    91  	default:
    92  		err := k.syncProducer.SendMessage(ctx, topic,
    93  			partitionNum, message.Key, message.Value)
    94  		return cerror.WrapError(cerror.ErrKafkaSendMessage, err)
    95  	}
    96  }
    97  
    98  func (k *kafkaDDLProducer) Close() {
    99  	// We have to hold the lock to prevent write to closed producer.
   100  	k.closedMu.Lock()
   101  	defer k.closedMu.Unlock()
   102  	// If the producer was already closed, we should skip the close operation.
   103  	if k.closed {
   104  		// We need to guard against double closed the clients,
   105  		// which could lead to panic.
   106  		log.Warn("Kafka DDL producer already closed",
   107  			zap.String("namespace", k.id.Namespace),
   108  			zap.String("changefeed", k.id.ID))
   109  		return
   110  	}
   111  	k.closed = true
   112  
   113  	if k.syncProducer != nil {
   114  		k.syncProducer.Close()
   115  	}
   116  }