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


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java:
##########
@@ -616,6 +620,90 @@ public void 
testCommitSyncTriggersFencedExceptionFromCommitAsync() {
         assertEquals("Get fenced exception for group.instance.id 
groupInstanceId1", e.getMessage());
     }
 
+    @Test
+    public void testCommitSyncAwaitsCommitAsyncCompletionWithEmptyOffsets() {
+        time = new MockTime(1);
+        consumer = newConsumer();
+
+        // Commit async (incomplete)
+        
doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+        
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        final TopicPartition tp = new TopicPartition("foo", 0);
+        consumer.assign(Collections.singleton(tp));
+        consumer.seek(tp, 20);
+        consumer.commitAsync();
+
+        // Commit async is not completed yet, so commit sync should wait for 
it to complete (time out)
+        assertThrows(TimeoutException.class, () -> 
consumer.commitSync(Collections.emptyMap(), Duration.ofMillis(100)));
+
+        // Complete async commit event
+        final ArgumentCaptor<AsyncCommitEvent> commitEventCaptor = 
ArgumentCaptor.forClass(AsyncCommitEvent.class);
+        verify(applicationEventHandler).add(commitEventCaptor.capture());
+        final AsyncCommitEvent commitEvent = commitEventCaptor.getValue();
+        commitEvent.future().complete(null);
+
+        // Commit async is completed, so commit sync completes immediately 
(since offsets are empty)
+        assertDoesNotThrow(() -> consumer.commitSync(Collections.emptyMap(), 
Duration.ofMillis(100)));
+    }
+
+    @Test
+    public void testCommitSyncAwaitsCommitAsyncCompletionWithNonEmptyOffsets() 
{
+        time = new MockTime(1);
+        consumer = newConsumer();
+
+        // Commit async (incomplete)
+        
doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+        
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        final TopicPartition tp = new TopicPartition("foo", 0);
+        consumer.assign(Collections.singleton(tp));
+        consumer.seek(tp, 20);
+        consumer.commitAsync();
+
+        // Mock to complete sync event
+        completeCommitSyncApplicationEventSuccessfully();
+
+        // Commit async is not completed yet, so commit sync should wait for 
it to complete (time out)
+        assertThrows(TimeoutException.class, () -> 
consumer.commitSync(Collections.singletonMap(tp, new OffsetAndMetadata(20)), 
Duration.ofMillis(100)));
+
+        // Complete async commit event and sync commit event
+        final ArgumentCaptor<AsyncCommitEvent> commitEventCaptor = 
ArgumentCaptor.forClass(AsyncCommitEvent.class);
+        verify(applicationEventHandler).add(commitEventCaptor.capture());
+        final AsyncCommitEvent commitEvent = commitEventCaptor.getValue();
+        commitEvent.future().complete(null);
+
+        // Commit async is completed, so commit sync completes immediately 
(since offsets are empty)
+        assertDoesNotThrow(() -> 
consumer.commitSync(Collections.singletonMap(tp, new OffsetAndMetadata(20)), 
Duration.ofMillis(100)));
+    }
+
+    @Test
+    public void testCommitSyncAwaitsCommitAsyncButDoesNotFail() {
+        time = new MockTime(1);
+        consumer = newConsumer();
+
+        // Commit async (incomplete)
+        
doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+        
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        final TopicPartition tp = new TopicPartition("foo", 0);
+        consumer.assign(Collections.singleton(tp));
+        consumer.seek(tp, 20);
+        consumer.commitAsync();
+
+        // Mock to complete sync event
+        completeCommitSyncApplicationEventSuccessfully();
+
+        // Commit async is not completed yet, so commit sync should wait for 
it to complete (time out)
+        assertThrows(TimeoutException.class, () -> 
consumer.commitSync(Collections.singletonMap(tp, new OffsetAndMetadata(20)), 
Duration.ofMillis(100)));

Review Comment:
   This section seems to be doing exactly the same as in the initial section of 
the previous test? maybe clearer to encapsulate in a common 
`testSyncCommitTimesoutAfterIncompleteAsyncCommit`, and then each tests 
proceeds completing the commits its way?



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