segmentio / kafka-go

Kafka library in Go
MIT License
7.61k stars 790 forks source link

ProtocolMetadata v1 without OwnedPartitions #1156

Open krvladislav opened 1 year ago

krvladislav commented 1 year ago

Describe the bug

JoinGroup request sends ProtocolMetadata marked as version v1 https://github.com/segmentio/kafka-go/blob/main/consumergroup.go#L998 but it never provides OwnedPartitions field which is expected by a protocol https://github.com/apache/kafka/blob/trunk/clients/src/main/resources/common/message/ConsumerProtocolSubscription.json#L34

This behavior can affect migration from kafka-go to another kafka client. Sarama provides guard https://github.com/Shopify/sarama/blob/main/consumer_group_members.go#L43 to skip this inconsistence silently. But franz-go throws an error which slowdown rebalance process.

Kafka Version

  • What version(s) of Kafka are you testing against? 2.6.0.0
  • What version of kafka-go are you using? v0.4.40

Expected Behavior

ProtocolMetadata must contains OwnedPartitions or sets Version as v0

aratz-lasa commented 1 year ago

I have more context that can help to understand further this issue.

Segmentio's consumer group metadata decoder is not compatible with Confluent (librdkafka) and Franz-go's format, which prevents running segmentio clients together with other clients.

The issue arises from Segmentio's failure to read and decode the latest extra fields of the consumer group metadata. In particular, it lacks compatibility with the format used by Confluent and Franzgo, making it unable to decode the metadata and compute the assignments if the Segmentio client is selected as the leader.

Brief overview of the problem

Segmentio does not read/decode the latest extra fields of the metadata (source: Franzgo Code).

After decoding, Segmentio checks whether there is any remaining byte, and if there is, it raises an error (source: Segmentio Code). The problem is that there will always be remaining bytes, because it does not read the latests fields.

This incompatibility results in Segmentio's inability to handle the metadata when it becomes the leader, causing problems in decoding and assignment computation.

Kafka Version

From 2.4.0 onwards. That's the version where KIP-429 was introduced and extended the metadata for the first time.

To Reproduce

It is as simple as running a segmentio consumer with librdkafka or franz-go consumer. As soon as the segment client is selected the leader, the segment consumer will shout an error and the consumer group will not progress.

The error will be: Failed to join group X: unable to read metadata for member, Y: %!w(<nil>)