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


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java:
##########
@@ -463,6 +467,42 @@ 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 = mkMap(mkEntry(tp, new 
OffsetAndMetadata(1)));
+        completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets);
+        
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        doReturn(cluster).when(metadata).fetch();
+
+        HashSet<String> topics = new HashSet<>();
+        topics.add(topicName);
+        doReturn(topics).when(cluster).topics();
+
+        consumer.subscribe(Pattern.compile("f*"));
+        verify(metadata).requestUpdateForNewTopics();
+        verify(subscriptions).matchesSubscribedPattern(topicName);
+
+        consumer.poll(Duration.ZERO);
+        clearInvocations(subscriptions);
+        verify(subscriptions, never()).matchesSubscribedPattern(topicName);

Review Comment:
   I would say that we should complete the story here to really know that our 
fix is kicking in. Up to this point we tested that we don't evaluate the regex 
on poll, but we should make sure it's because of the logic we're adding to skip 
it if metadata did not change. So what about extending the test to mock a 
metadata change, poll, and see the regex is indeed evaluated in that case? 
   
   ```suggestion
           when(metadata.updateVersion()).thenReturn(2);
           when(subscriptions.hasPatternSubscription()).thenReturn(true);
           consumer.poll(Duration.ZERO);
           verify(subscriptions).matchesSubscribedPattern(topicName);
   ```



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java:
##########
@@ -463,6 +467,42 @@ 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 = mkMap(mkEntry(tp, new 
OffsetAndMetadata(1)));
+        completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets);
+        
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        doReturn(cluster).when(metadata).fetch();
+
+        HashSet<String> topics = new HashSet<>();
+        topics.add(topicName);
+        doReturn(topics).when(cluster).topics();
+
+        consumer.subscribe(Pattern.compile("f*"));
+        verify(metadata).requestUpdateForNewTopics();
+        verify(subscriptions).matchesSubscribedPattern(topicName);
+
+        consumer.poll(Duration.ZERO);
+        clearInvocations(subscriptions);
+        verify(subscriptions, never()).matchesSubscribedPattern(topicName);
+    }

Review Comment:
   I would say that we should complete the story here to really know that our 
fix is kicking in. Up to this point we tested that we don't evaluate the regex 
on poll, but we should make sure it's because of the logic we're adding to skip 
it if metadata did not change. So what about extending the test to mock a 
metadata change, poll, and see the regex is indeed evaluated in that case? 
   
   ```suggestion
           when(metadata.updateVersion()).thenReturn(2);
           when(subscriptions.hasPatternSubscription()).thenReturn(true);
           consumer.poll(Duration.ZERO);
           verify(subscriptions).matchesSubscribedPattern(topicName);
   ```



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