github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/dmlproducer/kafka_dml_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 dmlproducer 15 16 import ( 17 "context" 18 "sync" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/failpoint" 22 "github.com/pingcap/log" 23 "github.com/pingcap/tiflow/cdc/model" 24 cerror "github.com/pingcap/tiflow/pkg/errors" 25 "github.com/pingcap/tiflow/pkg/sink/codec/common" 26 "github.com/pingcap/tiflow/pkg/sink/kafka" 27 "go.uber.org/zap" 28 ) 29 30 var _ DMLProducer = (*kafkaDMLProducer)(nil) 31 32 // kafkaDMLProducer is used to send messages to kafka. 33 type kafkaDMLProducer struct { 34 // id indicates which processor (changefeed) this sink belongs to. 35 id model.ChangeFeedID 36 // asyncProducer is used to send messages to kafka asynchronously. 37 asyncProducer kafka.AsyncProducer 38 // metricsCollector is used to report metrics. 39 metricsCollector kafka.MetricsCollector 40 // closedMu is used to protect `closed`. 41 // We need to ensure that closed producers are never written to. 42 closedMu sync.RWMutex 43 // closed is used to indicate whether the producer is closed. 44 // We also use it to guard against double closes. 45 closed bool 46 47 // failpointCh is used to inject failpoints to the run loop. Only used in test. 48 failpointCh chan error 49 50 cancel context.CancelFunc 51 } 52 53 // NewKafkaDMLProducer creates a new kafka producer. 54 func NewKafkaDMLProducer( 55 ctx context.Context, 56 changefeedID model.ChangeFeedID, 57 asyncProducer kafka.AsyncProducer, 58 metricsCollector kafka.MetricsCollector, 59 errCh chan error, 60 failpointCh chan error, 61 ) DMLProducer { 62 log.Info("Starting kafka DML producer ...", 63 zap.String("namespace", changefeedID.Namespace), 64 zap.String("changefeed", changefeedID.ID)) 65 66 ctx, cancel := context.WithCancel(ctx) 67 k := &kafkaDMLProducer{ 68 id: changefeedID, 69 asyncProducer: asyncProducer, 70 metricsCollector: metricsCollector, 71 closed: false, 72 failpointCh: failpointCh, 73 cancel: cancel, 74 } 75 76 // Start collecting metrics. 77 go k.metricsCollector.Run(ctx) 78 79 go func() { 80 if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled { 81 select { 82 case <-ctx.Done(): 83 return 84 case errCh <- err: 85 log.Error("Kafka DML producer run error", 86 zap.String("namespace", k.id.Namespace), 87 zap.String("changefeed", k.id.ID), 88 zap.Error(err)) 89 default: 90 log.Error("Error channel is full in kafka DML producer", 91 zap.String("namespace", k.id.Namespace), 92 zap.String("changefeed", k.id.ID), 93 zap.Error(err)) 94 } 95 } 96 }() 97 98 return k 99 } 100 101 func (k *kafkaDMLProducer) AsyncSendMessage( 102 ctx context.Context, topic string, 103 partition int32, message *common.Message, 104 ) error { 105 // We have to hold the lock to avoid writing to a closed producer. 106 // Close may be blocked for a long time. 107 k.closedMu.RLock() 108 defer k.closedMu.RUnlock() 109 110 // If the producer is closed, we should skip the message and return an error. 111 if k.closed { 112 return cerror.ErrKafkaProducerClosed.GenWithStackByArgs() 113 } 114 failpoint.Inject("KafkaSinkAsyncSendError", func() { 115 // simulate sending message to input channel successfully but flushing 116 // message to Kafka meets error 117 log.Info("KafkaSinkAsyncSendError error injected", zap.String("namespace", k.id.Namespace), 118 zap.String("changefeed", k.id.ID)) 119 k.failpointCh <- errors.New("kafka sink injected error") 120 failpoint.Return(nil) 121 }) 122 return k.asyncProducer.AsyncSend(ctx, topic, partition, 123 message.Key, message.Value, message.Callback) 124 } 125 126 func (k *kafkaDMLProducer) Close() { 127 // We have to hold the lock to synchronize closing with writing. 128 k.closedMu.Lock() 129 defer k.closedMu.Unlock() 130 // If the producer has already been closed, we should skip this close operation. 131 if k.closed { 132 // We need to guard against double closing the clients, 133 // which could lead to panic. 134 log.Warn("Kafka DML producer already closed", 135 zap.String("namespace", k.id.Namespace), 136 zap.String("changefeed", k.id.ID)) 137 return 138 } 139 if k.cancel != nil { 140 k.cancel() 141 } 142 143 close(k.failpointCh) 144 145 k.asyncProducer.Close() 146 k.closed = true 147 } 148 149 func (k *kafkaDMLProducer) run(ctx context.Context) error { 150 return k.asyncProducer.AsyncRunCallback(ctx) 151 }