-
Notifications
You must be signed in to change notification settings - Fork 3.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
rd_kafka_metadata can cause a UNKNOWN_TOPIC_OR_PART message #4589
Comments
On our program that wraps librdkafka, you can see this bad msg appearing prior to the successful subscription messages (first msg only appears when requesting metadata after subscription call) e.g.
|
This seems to happens every time when using rd_kafka_metadata after a valid subscription is made to an existing topic, where rd_kafka_queue_io_event_enable is used enable notifications (I need to use the file descriptor type notifications of items appearing in the queue). On a similar app that just does publishing, it doesnt seem to occur when rd_kafka_metadata called after publishing. |
I think this due to a bug discovered recently, affecting 2.1.0+ versions. librdkafka/src/rdkafka_metadata.c Line 1441 in 267367c
It's fixed in this PR: could you test if it solves your problem? |
Thanks, will let you know either way, once Ive tried. |
@emasab Bug still occurring when testing with #4660 Running my full program I still see the issue. Running with test program above, it still shows issue e.g.
Still same effect, that if I move the rd_kafka_metadata before the rd_kafka_subscribe, you dont see the problem...but doing a rd_kafka_metadata after a rd_kafka_subscribe always shows the problem. |
@emasab FYI: I tried librdkafka v2.0.2 & I dont see the bug occurring with that version. |
Could you check latest pushed commit in same PR? I've reproduced and fixed your case, will add a new test too. What happens is that when you call subscribe, the cache isn't populated until it joins the group, but if in the meantime it receives application metadata calls, it shouldn't update the group, because that causes this error. |
Yeah, there were changes with KIP 320 in 2.1.0 |
@emasab looks ok with latest change on PR , have tested again with original app. |
Description
Using a client with
rd_kafka_queue_io_event_enable
that is a consumer usingrd_kafka_subscribe
.This works fine until metadata is requested via
rd_kafka_metadata
.If the metadata is requested any time after making the subscription, the poll returns a message with error set to UNKNOWN_TOPIC_OR_PART (offset -1001) for every topic subscribed to (although the metadata returns correct results and the subscription continues getting messages).
If the metadata is requested prior to making any subscription, I dont get an error message from the poll.
How to reproduce
example client program (given topic exists called
topic1
with partition0
, brokerlocalhost:9092
) purely to show the error message.gcc main.c -lrdkafka -lz -lpthread -lssl
outputs
if you edit the code to put the metadata request before the subscription, the error msg no longer appears.
The debug log gets a
TOPICERR
entry only when metadata requested after subscription.Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
2.3.0
7.0.1
see example above
OSX(x64)>
debug=..
as necessary) from librdkafkaThe text was updated successfully, but these errors were encountered: