From bba3a038973ddf1d74c21d2973306bf0acb94391 Mon Sep 17 00:00:00 2001 From: YYChildren <1946231687@qq.com> Date: Tue, 5 Aug 2025 08:59:17 +0800 Subject: [PATCH] fix: When appRebalanceEnable == true and rebalanceCb == nil and channel == nil, it shound return ev:https://github.com/confluentinc/confluent-kafka-go/issues/1457 --- kafka/consumer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer.go b/kafka/consumer.go index efc54d00e..51a4d2d03 100644 --- a/kafka/consumer.go +++ b/kafka/consumer.go @@ -1026,7 +1026,7 @@ func (c *Consumer) handleRebalanceEvent(channel chan Event, rkev *C.rd_kafka_eve } - if channel != nil && c.appRebalanceEnable && c.rebalanceCb == nil { + if c.appRebalanceEnable && c.rebalanceCb == nil { // Channel-based consumer with rebalancing enabled, // return the rebalance event and rely on the application // to call *Assign() / *Unassign().