lianetm commented on code in PR #20521:
URL: https://github.com/apache/kafka/pull/20521#discussion_r2408144601


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java:
##########
@@ -487,6 +488,16 @@ public void onSubscriptionUpdated() {
      * active call to {@link 
org.apache.kafka.clients.consumer.KafkaConsumer#poll(Duration)}"
      */
     public void onConsumerPoll() {
+        if (subscriptions.hasPatternSubscription()) {
+            final Set<String> topicsToSubscribe = 
metadata.fetch().topics().stream()
+                .filter(subscriptions::matchesSubscribedPattern)
+                .collect(Collectors.toSet());
+            if (subscriptions.subscribeFromPattern(topicsToSubscribe)) {
+                metadata.requestUpdateForNewTopics();
+            }
+            subscriptionUpdated.compareAndSet(false, true);
+        }

Review Comment:
   uhm this needs to happen only if the metadata changed (and it's expensive, 
regex eval against all topics).
   Like done here:
   
https://github.com/apache/kafka/blob/f5a87b37030708e877d40169da9b64a1e382c4f3/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java#L415
   
   Given that there is already a `metadataVersionSnapshot` for that purpose but 
at the `ApplicationEventProcessor` level, we could probably consider having a 
separate func here to `updatePatternSubscription`, and then call it from the 
`ApplicationEventProcessor` only if metadata changed (right before calling this 
onConsumerPoll 
https://github.com/apache/kafka/blob/f5a87b37030708e877d40169da9b64a1e382c4f3/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java#L232)



-- 
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]

Reply via email to