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


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java:
##########
@@ -463,6 +466,44 @@ public void onPartitionsAssigned(final 
Collection<TopicPartition> partitions) {
         assertTrue(callbackExecuted.get());
     }
 
+    @Test
+    public void testCheckForNewTopicOnlyWhenMetadataChange() {
+        SubscriptionState subscriptions = mock(SubscriptionState.class);
+        Cluster cluster = mock(Cluster.class);
+
+        consumer = newConsumer(
+                mock(FetchBuffer.class),
+                mock(ConsumerInterceptors.class),
+                mock(ConsumerRebalanceListenerInvoker.class),
+                subscriptions,
+                singletonList(new RoundRobinAssignor()),
+                "group-id",
+                "client-id");
+
+        final String topicName = "foo";
+        final int partition = 3;
+        final TopicPartition tp = new TopicPartition(topicName, partition);
+        
doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+        Map<TopicPartition, OffsetAndMetadata> offsets = 
Collections.singletonMap(tp, new OffsetAndMetadata(1));
+        completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets);
+        
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        doReturn(cluster).when(metadata).fetch();
+        doReturn(Collections.singleton(topicName)).when(cluster).topics();
+
+        consumer.subscribe(Pattern.compile("f*"));
+        verify(metadata).requestUpdateForNewTopics();
+        verify(subscriptions).matchesSubscribedPattern(topicName);
+        clearInvocations(subscriptions);
+
+        consumer.poll(Duration.ZERO);

Review Comment:
   ```suggestion
           when(subscriptions.hasPatternSubscription()).thenReturn(true);
           consumer.poll(Duration.ZERO);
   ```
   
   Let's ensure that the poll knows it has pattern subscription (so that we 
know that nothing is blocking the way to eval the regex on ln 1976 of the 
AsyncConsumer. Just the metadata version check we're testing blocks it). This 
is what will make sure that if we loose the check by mistake, the test will 
actually fail.  



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to