lucasbru commented on code in PR #15613: URL: https://github.com/apache/kafka/pull/15613#discussion_r1572360676
########## core/src/test/scala/integration/kafka/api/PlaintextConsumerCommitTest.scala: ########## @@ -304,6 +304,64 @@ class PlaintextConsumerCommitTest extends AbstractConsumerTest { consumeAndVerifyRecords(consumer = otherConsumer, numRecords = 1, startingOffset = 5, startingTimestamp = startingTimestamp) } + // TODO: This only works in the new consumer, but should be fixed for the old consumer as well + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly")) + def testCommitAsyncCompletedBeforeConsumerCloses(quorum: String, groupProtocol: String): Unit = { + // This is testing the contract that asynchronous offset commit are completed before the consumer + // is closed, even when no commit sync is performed as part of the close (due to auto-commit + // disabled, or simply because there no consumed offsets). + val producer = createProducer() + sendRecords(producer, numRecords = 3, tp) + sendRecords(producer, numRecords = 3, tp2) + + val consumer = createConsumer() + consumer.assign(List(tp, tp2).asJava) + + // Try without looking up the coordinator first + val cb = new CountConsumerCommitCallback + consumer.commitAsync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(1L))).asJava, cb) + consumer.commitAsync(Map[TopicPartition, OffsetAndMetadata]((tp2, new OffsetAndMetadata(1L))).asJava, cb) + consumer.close() + assertEquals(2, cb.successCount) + } + + // TODO: This only works in the new consumer, but should be fixed for the old consumer as well + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly")) + def testCommitAsyncCompletedBeforeCommitSyncReturns(quorum: String, groupProtocol: String): Unit = { + // This is testing the contract that asynchronous offset commits sent previously with the + // `commitAsync` are guaranteed to have their callbacks invoked prior to completion of + // `commitSync` (given that it does not time out). + val producer = createProducer() + sendRecords(producer, numRecords = 3, tp) + sendRecords(producer, numRecords = 3, tp2) + + val consumer = createConsumer() + consumer.assign(List(tp, tp2).asJava) + + // Try without looking up the coordinator first Review Comment: Okay got it. Yeah, the state "async commit pending because I do not know the coordinator yet" and the state "async commit is already sent to the coordinator" may cover two different cases. In fact, in the legacy consumer, these are two independent states the async commit can be in. In the async consumer as well, although waiting for the async commit to complete works independently of that state (since the different states are handled by the background thread, and we simplify wait for the future in the foreground thread). ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java: ########## @@ -616,6 +634,85 @@ public void testCommitSyncTriggersFencedExceptionFromCommitAsync() { assertEquals("Get fenced exception for group.instance.id groupInstanceId1", e.getMessage()); } + @Test + public void testCommitSyncAwaitsCommitAsyncCompletionWithEmptyOffsets() { + final TopicPartition tp = new TopicPartition("foo", 0); + testIncompleteAsyncCommit(tp); + + final CompletableFuture<Void> asyncCommitFuture = getLastEnqueuedEventFuture(); + + // 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 exceptionally async commit event + asyncCommitFuture.completeExceptionally(new KafkaException("Test exception")); + + // 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() { + final TopicPartition tp = new TopicPartition("foo", 0); + testIncompleteAsyncCommit(tp); + + final CompletableFuture<Void> asyncCommitFuture = getLastEnqueuedEventFuture(); + + // 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 exceptionally async commit event + asyncCommitFuture.completeExceptionally(new KafkaException("Test exception")); + + // Commit async is completed, so commit sync does not need to wait before committing its offsets + assertDoesNotThrow(() -> consumer.commitSync(Collections.singletonMap(tp, new OffsetAndMetadata(20)), Duration.ofMillis(100))); + } + + @Test + public void testCommitSyncAwaitsCommitAsyncButDoesNotFail() { + final TopicPartition tp = new TopicPartition("foo", 0); + testIncompleteAsyncCommit(tp); + + final CompletableFuture<Void> asyncCommitFuture = getLastEnqueuedEventFuture(); + + // 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 exceptionally async commit event + asyncCommitFuture.completeExceptionally(new KafkaException("Test exception")); + + // Commit async is completed exceptionally, but this will be handled by commit callback - commit sync should not fail. + assertDoesNotThrow(() -> consumer.commitSync(Collections.singletonMap(tp, new OffsetAndMetadata(20)), Duration.ofMillis(100))); + } + + private void testIncompleteAsyncCommit(TopicPartition tp) { + time = new MockTime(1); + consumer = newConsumer(); + + // Commit async (incomplete) + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Review Comment: No. Removed ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java: ########## @@ -616,6 +634,85 @@ public void testCommitSyncTriggersFencedExceptionFromCommitAsync() { assertEquals("Get fenced exception for group.instance.id groupInstanceId1", e.getMessage()); } + @Test + public void testCommitSyncAwaitsCommitAsyncCompletionWithEmptyOffsets() { + final TopicPartition tp = new TopicPartition("foo", 0); + testIncompleteAsyncCommit(tp); + + final CompletableFuture<Void> asyncCommitFuture = getLastEnqueuedEventFuture(); + + // 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 exceptionally async commit event + asyncCommitFuture.completeExceptionally(new KafkaException("Test exception")); + + // 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() { + final TopicPartition tp = new TopicPartition("foo", 0); + testIncompleteAsyncCommit(tp); + + final CompletableFuture<Void> asyncCommitFuture = getLastEnqueuedEventFuture(); + + // 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 exceptionally async commit event + asyncCommitFuture.completeExceptionally(new KafkaException("Test exception")); + + // Commit async is completed, so commit sync does not need to wait before committing its offsets + assertDoesNotThrow(() -> consumer.commitSync(Collections.singletonMap(tp, new OffsetAndMetadata(20)), Duration.ofMillis(100))); + } + + @Test + public void testCommitSyncAwaitsCommitAsyncButDoesNotFail() { Review Comment: Good catch. That's a regression after a refactor due to an earlier comment. The above test is supposed to complete the future successfully. ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java: ########## @@ -1005,6 +1102,43 @@ public void testNoWakeupInCloseCommit() { assertFalse(capturedEvent.get().future().isCompletedExceptionally()); } + @Test + public void testCloseAwaitPendingAsyncCommitIncomplete() { + 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(); + Exception e = assertThrows(KafkaException.class, () -> consumer.close(Duration.ofMillis(10))); + assertInstanceOf(TimeoutException.class, e.getCause()); + } + + @Test + public void testCloseAwaitPendingAsyncCommitComplete() { + time = new MockTime(1); + consumer = newConsumer(); + MockCommitCallback cb = new MockCommitCallback(); + + // Commit async (complete) + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Review Comment: No. Removed. ########## core/src/test/scala/integration/kafka/api/PlaintextConsumerCommitTest.scala: ########## @@ -304,6 +304,64 @@ class PlaintextConsumerCommitTest extends AbstractConsumerTest { consumeAndVerifyRecords(consumer = otherConsumer, numRecords = 1, startingOffset = 5, startingTimestamp = startingTimestamp) } + // TODO: This only works in the new consumer, but should be fixed for the old consumer as well + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly")) + def testCommitAsyncCompletedBeforeConsumerCloses(quorum: String, groupProtocol: String): Unit = { + // This is testing the contract that asynchronous offset commit are completed before the consumer + // is closed, even when no commit sync is performed as part of the close (due to auto-commit + // disabled, or simply because there no consumed offsets). Review Comment: Done ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java: ########## @@ -1005,6 +1102,43 @@ public void testNoWakeupInCloseCommit() { assertFalse(capturedEvent.get().future().isCompletedExceptionally()); } + @Test + public void testCloseAwaitPendingAsyncCommitIncomplete() { + time = new MockTime(1); + consumer = newConsumer(); + + // Commit async (incomplete) + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Review Comment: No. Removed. -- 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