You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
I'm using kcat as kafka client, when running below command in shell kcat -b 127.0.0.1:9092 -L -d metadata -X allow.auto.create.topics=true
the debug output shows that client will generate two metadata requests:
I captured packets with wireshark and check those two meta requests, and found the second metadata request for all topics has an incorrect topics list length, below is the capture screenshot,
the first one:
the second one:
As we can see the second metadata request use a int32 to describe the topics length(0x 00 00 00 00), while the first metadata request use the correct 1-based uvarint(0x 01).
I checked the source code, and see the difference may be caused by https://github.com/confluentinc/librdkafka/blob/master/src/rdkafka_request.c#L2803-L2826
for the case topic_cnt == 0, we don't do varint encoding.
It seems for the same API version(Metadata Request Version 12), there's a mismatch behavior of encoding.
I think for the topic_cnt == 0 && ApiVersion >= 9 case, we should also invoke rd_kafka_buf_finalize_arraycnt to do varint encoding.
The text was updated successfully, but these errors were encountered:
ProBrian
added a commit
to ProBrian/librdkafka
that referenced
this issue
Apr 25, 2025
Context
I'm using kcat as kafka client, when running below command in shell
kcat -b 127.0.0.1:9092 -L -d metadata -X allow.auto.create.topics=true
the debug output shows that client will generate two metadata requests:
I captured packets with wireshark and check those two meta requests, and found the second metadata request for all topics has an incorrect topics list length, below is the capture screenshot,
the first one:
the second one:
As we can see the second metadata request use a int32 to describe the topics length(0x 00 00 00 00), while the first metadata request use the correct 1-based uvarint(0x 01).
I checked the source code, and see the difference may be caused by https://github.com/confluentinc/librdkafka/blob/master/src/rdkafka_request.c#L2803-L2826
for the case
topic_cnt == 0
, we don't do varint encoding.It seems for the same API version(Metadata Request Version 12), there's a mismatch behavior of encoding.
I think for the
topic_cnt == 0 && ApiVersion >= 9
case, we should also invokerd_kafka_buf_finalize_arraycnt
to do varint encoding.The text was updated successfully, but these errors were encountered: