librdkafka icon indicating copy to clipboard operation
librdkafka copied to clipboard

Metadata request doesn't handle topics length correctly

Open ProBrian opened this issue 6 months ago • 0 comments

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:

%7|1745481050.866|METADATA|rdkafka#producer-1| [thrd:127.0.0.1:9092/bootstrap]: 127.0.0.1:9092/bootstrap: Request metadata for brokers only: connected %7|1745481050.866|METADATA|rdkafka#producer-1| [thrd:app]: 127.0.0.1:9092/bootstrap: Request metadata for all topics: application requested %7|1745481050.867|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: ===== Received metadata: connected ===== %7|1745481050.867|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: ClusterId: 5L6g3nShT-eMCtK--X86sw, ControllerId: 1 %7|1745481050.867|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: 1 brokers, 0 topics %7|1745481050.867|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: Broker #0/1: localhost:9092 NodeId 1 %7|1745481050.868|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: ===== Received metadata: application requested ===== %7|1745481050.868|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: ClusterId: 5L6g3nShT-eMCtK--X86sw, ControllerId: 1 %7|1745481050.868|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: 1 brokers, 2 topics %7|1745481050.868|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: Broker #0/1: localhost:9092 NodeId 1 %7|1745481050.868|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: Topic test-topic with 1 partitions %7|1745481050.868|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: Topic __consumer_offsets with 50 partitions %7|1745481050.868|METADATA|rdkafka#producer-1| [thrd:main]: 127.0.0.1:9092/bootstrap: Caching full metadata with 1 broker(s) and 2 topic(s): application requested

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:

Image

the second one:

Image

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.

ProBrian avatar Apr 24 '25 09:04 ProBrian