lianetm commented on code in PR #15585:
URL: https://github.com/apache/kafka/pull/15585#discussion_r1538158322
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1750,8 +1753,14 @@ private void subscribeInternal(Pattern pattern,
Optional<ConsumerRebalanceListen
throwIfNoAssignorsConfigured();
log.info("Subscribed to pattern: '{}'", pattern);
subscriptions.subscribe(pattern, listener);
- updatePatternSubscription(metadata.fetch());
metadata.requestUpdateForNewTopics();
+ Cluster cache = metadata.fetch();
+
+ while (cache == metadata.fetch()) {
+ log.info("Waiting for new metadata update");
+ }
Review Comment:
I guess this is with the purpose of ensuring we get metadata before we
proceed to actually join the group with the resolved list of topic names. But
the way it is I think is not conceptually right. With this we're only ensuring
that there is a metadata, but that does not ensure that we have discovered any
topic that matches the regex (I expect that in that case we would come out of
the while, and the `SubscriptionChangeEvent` would lead to a first HB to join
the group but with empty topic list, which would be an invalid first HB
request).
I wonder if we could simply reuse the logic that we already have in
`updatePatternSubscription`, that knows when topics matching the regex are
discovered, and it's only at that point that we generate the event and join the
group? We would end up with a much simpler logic here (only needing to
`subscribe`, `updatePatternSubscription` and `requestUpdateForNewTopics` like
it was before), and we would enqueue the subscription change event only when we
actually discover topics
([here](https://github.com/apache/kafka/blob/f8ce7feebcede6c6da93c649413a64695bc8d4c1/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java#L1466))
. What do you think?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]