github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/dmlproducer/pulsar_dml_producer.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 dmlproducer
    15  
    16  import (
    17  	"context"
    18  	"encoding/json"
    19  	"sync"
    20  	"time"
    21  
    22  	"github.com/apache/pulsar-client-go/pulsar"
    23  	lru "github.com/hashicorp/golang-lru"
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/failpoint"
    26  	"github.com/pingcap/log"
    27  	"github.com/pingcap/tiflow/cdc/model"
    28  	"github.com/pingcap/tiflow/cdc/sink/metrics/mq"
    29  	"github.com/pingcap/tiflow/pkg/config"
    30  	cerror "github.com/pingcap/tiflow/pkg/errors"
    31  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    32  	"go.uber.org/zap"
    33  )
    34  
    35  var _ DMLProducer = (*pulsarDMLProducer)(nil)
    36  
    37  // pulsarDMLProducer is used to send messages to pulsar.
    38  type pulsarDMLProducer struct {
    39  	// id indicates which processor (changefeed) this sink belongs to.
    40  	id model.ChangeFeedID
    41  	// We hold the client to make close operation faster.
    42  	// Please see the comment of Close().
    43  	client pulsar.Client
    44  	// producers is used to send messages to pulsar.
    45  	// One topic only use one producer , so we want to have many topics but use less memory,
    46  	// lru is a good idea to solve this question.
    47  	// support multiple topics
    48  	producers *lru.Cache
    49  
    50  	// closedMu is used to protect `closed`.
    51  	// We need to ensure that closed producers are never written to.
    52  	closedMu sync.RWMutex
    53  	// closed is used to indicate whether the producer is closed.
    54  	// We also use it to guard against double closes.
    55  	closed bool
    56  
    57  	// failpointCh is used to inject failpoints to the run loop.
    58  	// Only used in test.
    59  	failpointCh chan error
    60  	// closeCh is send error
    61  	errChan chan error
    62  
    63  	pConfig *config.PulsarConfig
    64  }
    65  
    66  // NewPulsarDMLProducer creates a new pulsar producer.
    67  func NewPulsarDMLProducer(
    68  	ctx context.Context,
    69  	changefeedID model.ChangeFeedID,
    70  	client pulsar.Client,
    71  	sinkConfig *config.SinkConfig,
    72  	errCh chan error,
    73  	failpointCh chan error,
    74  ) (DMLProducer, error) {
    75  	log.Info("Creating pulsar DML producer ...",
    76  		zap.String("namespace", changefeedID.Namespace),
    77  		zap.String("changefeed", changefeedID.ID))
    78  	start := time.Now()
    79  
    80  	var pulsarConfig *config.PulsarConfig
    81  	if sinkConfig.PulsarConfig == nil {
    82  		log.Error("new pulsar DML producer fail,sink:pulsar config is empty")
    83  		return nil, cerror.ErrPulsarInvalidConfig.
    84  			GenWithStackByArgs("pulsar config is empty")
    85  	}
    86  
    87  	pulsarConfig = sinkConfig.PulsarConfig
    88  	defaultTopicName := pulsarConfig.GetDefaultTopicName()
    89  	defaultProducer, err := newProducer(pulsarConfig, client, defaultTopicName)
    90  	if err != nil {
    91  		go client.Close()
    92  		return nil, cerror.WrapError(cerror.ErrPulsarNewProducer, err)
    93  	}
    94  	producerCacheSize := config.DefaultPulsarProducerCacheSize
    95  	if pulsarConfig != nil && pulsarConfig.PulsarProducerCacheSize != nil {
    96  		producerCacheSize = int(*pulsarConfig.PulsarProducerCacheSize)
    97  	}
    98  
    99  	producers, err := lru.NewWithEvict(producerCacheSize, func(key interface{}, value interface{}) {
   100  		// this is call when lru Remove producer or auto remove producer
   101  		pulsarProducer, ok := value.(pulsar.Producer)
   102  		if ok && pulsarProducer != nil {
   103  			pulsarProducer.Close()
   104  		}
   105  	})
   106  	if err != nil {
   107  		go client.Close()
   108  		return nil, cerror.WrapError(cerror.ErrPulsarNewProducer, err)
   109  	}
   110  
   111  	producers.Add(defaultTopicName, defaultProducer)
   112  
   113  	p := &pulsarDMLProducer{
   114  		id:          changefeedID,
   115  		client:      client,
   116  		producers:   producers,
   117  		pConfig:     pulsarConfig,
   118  		closed:      false,
   119  		failpointCh: failpointCh,
   120  		errChan:     errCh,
   121  	}
   122  	log.Info("Pulsar DML producer created", zap.Stringer("changefeed", p.id),
   123  		zap.Duration("duration", time.Since(start)))
   124  	return p, nil
   125  }
   126  
   127  // AsyncSendMessage  Async send one message
   128  func (p *pulsarDMLProducer) AsyncSendMessage(
   129  	ctx context.Context, topic string,
   130  	partition int32, message *common.Message,
   131  ) error {
   132  	wrapperSchemaAndTopic(message)
   133  
   134  	// We have to hold the lock to avoid writing to a closed producer.
   135  	// Close may be blocked for a long time.
   136  	p.closedMu.RLock()
   137  	defer p.closedMu.RUnlock()
   138  
   139  	// If producers are closed, we should skip the message and return an error.
   140  	if p.closed {
   141  		return cerror.ErrPulsarProducerClosed.GenWithStackByArgs()
   142  	}
   143  	failpoint.Inject("PulsarSinkAsyncSendError", func() {
   144  		// simulate sending message to input channel successfully but flushing
   145  		// message to Pulsar meets error
   146  		log.Info("PulsarSinkAsyncSendError error injected", zap.String("namespace", p.id.Namespace),
   147  			zap.String("changefeed", p.id.ID))
   148  		p.failpointCh <- errors.New("pulsar sink injected error")
   149  		failpoint.Return(nil)
   150  	})
   151  	data := &pulsar.ProducerMessage{
   152  		Payload: message.Value,
   153  		Key:     message.GetPartitionKey(),
   154  	}
   155  
   156  	producer, err := p.GetProducerByTopic(topic)
   157  	if err != nil {
   158  		return err
   159  	}
   160  
   161  	// if for stress test record , add count to message callback function
   162  
   163  	producer.SendAsync(ctx, data,
   164  		func(id pulsar.MessageID, m *pulsar.ProducerMessage, err error) {
   165  			// fail
   166  			if err != nil {
   167  				e := cerror.WrapError(cerror.ErrPulsarAsyncSendMessage, err)
   168  				log.Error("Pulsar DML producer async send error",
   169  					zap.String("namespace", p.id.Namespace),
   170  					zap.String("changefeed", p.id.ID),
   171  					zap.Int("messageSize", len(m.Payload)),
   172  					zap.String("topic", topic),
   173  					zap.String("schema", message.GetSchema()),
   174  					zap.Error(err))
   175  				mq.IncPublishedDMLFail(topic, p.id.ID, message.GetSchema())
   176  				// use this select to avoid send error to a closed channel
   177  				// the ctx will always be called before the errChan is closed
   178  				select {
   179  				case <-ctx.Done():
   180  					return
   181  				case p.errChan <- e:
   182  				default:
   183  					log.Warn("Error channel is full in pulsar DML producer",
   184  						zap.Stringer("changefeed", p.id), zap.Error(e))
   185  				}
   186  			} else if message.Callback != nil {
   187  				// success
   188  				message.Callback()
   189  				mq.IncPublishedDMLSuccess(topic, p.id.ID, message.GetSchema())
   190  			}
   191  		})
   192  
   193  	mq.IncPublishedDMLCount(topic, p.id.ID, message.GetSchema())
   194  
   195  	return nil
   196  }
   197  
   198  func (p *pulsarDMLProducer) Close() { // We have to hold the lock to synchronize closing with writing.
   199  	p.closedMu.Lock()
   200  	defer p.closedMu.Unlock()
   201  	// If the producer has already been closed, we should skip this close operation.
   202  	if p.closed {
   203  		// We need to guard against double closing the clients,
   204  		// which could lead to panic.
   205  		log.Warn("Pulsar DML producer already closed",
   206  			zap.String("namespace", p.id.Namespace),
   207  			zap.String("changefeed", p.id.ID))
   208  		return
   209  	}
   210  	close(p.failpointCh)
   211  	p.closed = true
   212  	start := time.Now()
   213  	keys := p.producers.Keys()
   214  	for _, topic := range keys {
   215  		p.producers.Remove(topic) // callback func will be called
   216  		topicName, _ := topic.(string)
   217  		log.Info("Async client closed in pulsar DML producer",
   218  			zap.Duration("duration", time.Since(start)),
   219  			zap.String("namespace", p.id.Namespace),
   220  			zap.String("changefeed", p.id.ID), zap.String("topic", topicName))
   221  	}
   222  	p.client.Close()
   223  }
   224  
   225  // newProducer creates a pulsar producer
   226  // One topic is used by one producer
   227  func newProducer(
   228  	pConfig *config.PulsarConfig,
   229  	client pulsar.Client,
   230  	topicName string,
   231  ) (pulsar.Producer, error) {
   232  	maxReconnectToBroker := uint(config.DefaultMaxReconnectToPulsarBroker)
   233  	option := pulsar.ProducerOptions{
   234  		Topic:                topicName,
   235  		MaxReconnectToBroker: &maxReconnectToBroker,
   236  	}
   237  	if pConfig.BatchingMaxMessages != nil {
   238  		option.BatchingMaxMessages = *pConfig.BatchingMaxMessages
   239  	}
   240  	if pConfig.BatchingMaxPublishDelay != nil {
   241  		option.BatchingMaxPublishDelay = pConfig.BatchingMaxPublishDelay.Duration()
   242  	}
   243  	if pConfig.CompressionType != nil {
   244  		option.CompressionType = pConfig.CompressionType.Value()
   245  		option.CompressionLevel = pulsar.Default
   246  	}
   247  	if pConfig.SendTimeout != nil {
   248  		option.SendTimeout = pConfig.SendTimeout.Duration()
   249  	}
   250  
   251  	producer, err := client.CreateProducer(option)
   252  	if err != nil {
   253  		return nil, err
   254  	}
   255  
   256  	log.Info("create pulsar producer success", zap.String("topic", topicName))
   257  
   258  	return producer, nil
   259  }
   260  
   261  func (p *pulsarDMLProducer) getProducer(topic string) (pulsar.Producer, bool) {
   262  	target, ok := p.producers.Get(topic)
   263  	if ok {
   264  		producer, ok := target.(pulsar.Producer)
   265  		if ok {
   266  			return producer, true
   267  		}
   268  	}
   269  	return nil, false
   270  }
   271  
   272  // GetProducerByTopic get producer by topicName,
   273  // if not exist, it will create a producer with topicName, and set in LRU cache
   274  // more meta info at pulsarDMLProducer's producers
   275  func (p *pulsarDMLProducer) GetProducerByTopic(topicName string) (producer pulsar.Producer, err error) {
   276  	getProducer, ok := p.getProducer(topicName)
   277  	if ok && getProducer != nil {
   278  		return getProducer, nil
   279  	}
   280  
   281  	if !ok { // create a new producer for the topicName
   282  		producer, err = newProducer(p.pConfig, p.client, topicName)
   283  		if err != nil {
   284  			return nil, err
   285  		}
   286  		p.producers.Add(topicName, producer)
   287  	}
   288  
   289  	return producer, nil
   290  }
   291  
   292  // wrapperSchemaAndTopic wrapper schema and topic
   293  func wrapperSchemaAndTopic(m *common.Message) {
   294  	if m.Schema == nil {
   295  		if m.Protocol == config.ProtocolMaxwell {
   296  			mx := &maxwellMessage{}
   297  			err := json.Unmarshal(m.Value, mx)
   298  			if err != nil {
   299  				log.Error("unmarshal maxwell message failed", zap.Error(err))
   300  				return
   301  			}
   302  			if len(mx.Database) > 0 {
   303  				m.Schema = &mx.Database
   304  			}
   305  			if len(mx.Table) > 0 {
   306  				m.Table = &mx.Table
   307  			}
   308  		}
   309  		if m.Protocol == config.ProtocolCanal { // canal protocol set multi schemas in one topic
   310  			m.Schema = str2Pointer("multi_schema")
   311  		}
   312  	}
   313  }
   314  
   315  // maxwellMessage is the message format of maxwell
   316  type maxwellMessage struct {
   317  	Database string `json:"database"`
   318  	Table    string `json:"table"`
   319  }
   320  
   321  // str2Pointer returns the pointer of the string.
   322  func str2Pointer(str string) *string {
   323  	return &str
   324  }