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
On consumer group (re)join the consumer does a metadata refresh using the full subscription list, (2 topics), but the periodic metadata refresher requests metadata for all known topics (rd_kafka_topic_t) in librdkafka. For the high-level consumer a known topic is one that has been referenced by the fetcher, thus a topic that has been assign:ed().
The problem here is that while you subscribe to two topics, your consumer is only receveing an assignment for a partition for one, so the fetcher only knows (cares is a better word) about that one topic.
Upon the next periodic metadata refresh it requests only the known topic, sees that the effective subscription list changed from 2 to 1 topics and starts the rejoin process.
The rejoin will trigger a proper metadata refresh for the full subscription list, so the consumer will indeed subscribe to two topics.
Wait until the next period refresh and the same thing happens again.
We'll need to figure out what the best approach is to fix this issue, but it probably boils down to requesting both known and subscribed topics.
This fix will not make it into v1.0.0.
The workaround is to use regex subscription for at least one topic, which causes the periodic metadata refresh to request all topics in the cluster.
Thus, in your cause, subscribe to: demo-topic-2 and ^demo-topic-2_delay$ (the latter being an exact regex pattern).
Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
librdkafka version (release number or git tag): all
The text was updated successfully, but these errors were encountered:
thank you for linking this and posting about it here
i wouldnt have find the workaround if it was just kept posted in issues of confluent-kafka-python repo only (because i'm searching this repo mostly for issues and discussions)
Bumping this topic as I think I face a similar issue. I am using the rust binding of librdkafka library (1.9.2).
I implemented a periodic resubscribe logic which subscribes to topics based on the input regex. This calls to rd_kafka_metadata API with no topics specified because I would like to filter based on my regex.
I compare the current assignment and the topics in the metadata and change the subscription if there is any difference.
However this casuses re-join during most of the fetch request. Which I don't particularly understand because the topics are not really changing to where I have been subscribed to.
What would be the preferred way to implement this behavior for a reasonable periodic resubscribe without having to do re-join all the time?
From confluentinc/confluent-kafka-python#559 :
On consumer group (re)join the consumer does a metadata refresh using the full subscription list, (2 topics), but the periodic metadata refresher requests metadata for all known topics (rd_kafka_topic_t) in librdkafka. For the high-level consumer a known topic is one that has been referenced by the fetcher, thus a topic that has been assign:ed().
The problem here is that while you subscribe to two topics, your consumer is only receveing an assignment for a partition for one, so the fetcher only knows (cares is a better word) about that one topic.
Upon the next periodic metadata refresh it requests only the known topic, sees that the effective subscription list changed from 2 to 1 topics and starts the rejoin process.
The rejoin will trigger a proper metadata refresh for the full subscription list, so the consumer will indeed subscribe to two topics.
Wait until the next period refresh and the same thing happens again.
We'll need to figure out what the best approach is to fix this issue, but it probably boils down to requesting both known and subscribed topics.
This fix will not make it into v1.0.0.
The workaround is to use regex subscription for at least one topic, which causes the periodic metadata refresh to request all topics in the cluster.
Thus, in your cause, subscribe to:
demo-topic-2
and^demo-topic-2_delay$
(the latter being an exact regex pattern).Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
The text was updated successfully, but these errors were encountered: