fix: kafka use shared channel to receive produce result (#38532)

issue: #38531

---------

Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
Zhen Ye 2024-12-17 23:10:45 +08:00 committed by GitHub
parent 4d5572d7ed
commit b18a3cfce2
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
3 changed files with 19 additions and 23 deletions

View File

@ -220,8 +220,7 @@ func (kc *kafkaClient) CreateProducer(ctx context.Context, options common.Produc
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds())) metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc() metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc()
deliveryChan := make(chan kafka.Event, 128) producer := &kafkaProducer{p: pp, stopCh: make(chan struct{}), topic: options.Topic}
producer := &kafkaProducer{p: pp, deliveryChan: deliveryChan, topic: options.Topic}
return producer, nil return producer, nil
} }

View File

@ -18,11 +18,11 @@ import (
) )
type kafkaProducer struct { type kafkaProducer struct {
p *kafka.Producer p *kafka.Producer
topic string topic string
deliveryChan chan kafka.Event closeOnce sync.Once
closeOnce sync.Once isClosed bool
isClosed bool stopCh chan struct{}
} }
func (kp *kafkaProducer) Topic() string { func (kp *kafkaProducer) Topic() string {
@ -44,24 +44,28 @@ func (kp *kafkaProducer) Send(ctx context.Context, message *mqcommon.ProducerMes
header := kafka.Header{Key: key, Value: []byte(value)} header := kafka.Header{Key: key, Value: []byte(value)}
headers = append(headers, header) headers = append(headers, header)
} }
resultCh := make(chan kafka.Event, 1)
err := kp.p.Produce(&kafka.Message{ err := kp.p.Produce(&kafka.Message{
TopicPartition: kafka.TopicPartition{Topic: &kp.topic, Partition: mqwrapper.DefaultPartitionIdx}, TopicPartition: kafka.TopicPartition{Topic: &kp.topic, Partition: mqwrapper.DefaultPartitionIdx},
Value: message.Payload, Value: message.Payload,
Headers: headers, Headers: headers,
}, kp.deliveryChan) }, resultCh)
if err != nil { if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc() metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
return nil, err return nil, err
} }
e, ok := <-kp.deliveryChan var m *kafka.Message
if !ok { select {
case <-kp.stopCh:
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc() metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
log.Error("kafka produce message fail because of delivery chan is closed", zap.String("topic", kp.topic)) log.Error("kafka produce message fail because of kafka producer is closed", zap.String("topic", kp.topic))
return nil, common.NewIgnorableError(fmt.Errorf("delivery chan of kafka producer is closed")) return nil, common.NewIgnorableError(fmt.Errorf("kafka producer is closed"))
case e := <-resultCh:
m = e.(*kafka.Message)
} }
m := e.(*kafka.Message)
if m.TopicPartition.Error != nil { if m.TopicPartition.Error != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc() metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
return nil, m.TopicPartition.Error return nil, m.TopicPartition.Error
@ -86,8 +90,7 @@ func (kp *kafkaProducer) Close() {
log.Warn("There are still un-flushed outstanding events", zap.Int("event_num", i), zap.String("topic", kp.topic)) log.Warn("There are still un-flushed outstanding events", zap.Int("event_num", i), zap.String("topic", kp.topic))
} }
close(kp.deliveryChan) close(kp.stopCh)
cost := time.Since(start).Milliseconds() cost := time.Since(start).Milliseconds()
if cost > 500 { if cost > 500 {
log.Debug("kafka producer is closed", zap.String("topic", kp.topic), zap.Int64("time cost(ms)", cost)) log.Debug("kafka producer is closed", zap.String("topic", kp.topic), zap.Int64("time cost(ms)", cost))

View File

@ -7,7 +7,6 @@ import (
"testing" "testing"
"time" "time"
"github.com/cockroachdb/errors"
"github.com/confluentinc/confluent-kafka-go/kafka" "github.com/confluentinc/confluent-kafka-go/kafka"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -44,26 +43,21 @@ func TestKafkaProducer_SendSuccess(t *testing.T) {
func TestKafkaProducer_SendFail(t *testing.T) { func TestKafkaProducer_SendFail(t *testing.T) {
kafkaAddress := getKafkaBrokerList() kafkaAddress := getKafkaBrokerList()
{ {
deliveryChan := make(chan kafka.Event, 1)
rand.Seed(time.Now().UnixNano()) rand.Seed(time.Now().UnixNano())
topic := fmt.Sprintf("test-topic-%d", rand.Int()) topic := fmt.Sprintf("test-topic-%d", rand.Int())
pp, err := kafka.NewProducer(&kafka.ConfigMap{"bootstrap.servers": kafkaAddress}) pp, err := kafka.NewProducer(&kafka.ConfigMap{"bootstrap.servers": kafkaAddress})
assert.NoError(t, err) assert.NoError(t, err)
producer := &kafkaProducer{p: pp, deliveryChan: deliveryChan, topic: topic} producer := &kafkaProducer{p: pp, stopCh: make(chan struct{}), topic: topic}
close(producer.stopCh)
msg := &common.ProducerMessage{ msg := &common.ProducerMessage{
Payload: []byte{1}, Payload: []byte{1},
Properties: map[string]string{}, Properties: map[string]string{},
} }
var resultMsg kafka.Event = &kafka.Message{TopicPartition: kafka.TopicPartition{Error: errors.New("error")}}
deliveryChan <- resultMsg
ret, err := producer.Send(context.TODO(), msg) ret, err := producer.Send(context.TODO(), msg)
assert.Nil(t, ret) assert.Nil(t, ret)
assert.Error(t, err) assert.Error(t, err)
producer.Close()
} }
} }