kirktrue commented on code in PR #16031:
URL: https://github.com/apache/kafka/pull/16031#discussion_r1626752933


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java:
##########
@@ -613,15 +614,44 @@ public void testOffsetFetchRequestErroredRequests(final 
Errors error, final bool
         }
     }
 
+    @ParameterizedTest
+    @MethodSource("offsetFetchExceptionSupplier")
+    public void testOffsetFetchRequestTimeoutRequests(final Errors error,
+                                                      final boolean 
isRetriable) {
+        CommitRequestManager commitRequestManager = create(true, 100);
+        
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
+
+        Set<TopicPartition> partitions = new HashSet<>();
+        partitions.add(new TopicPartition("t1", 0));
+        List<CompletableFuture<Map<TopicPartition, OffsetAndMetadata>>> 
futures = sendAndVerifyDuplicatedOffsetFetchRequests(
+                commitRequestManager,
+                partitions,
+                1,
+                error);
+
+        if (isRetriable) {
+            futures.forEach(f -> assertFalse(f.isDone()));
+
+            // Insert a long enough sleep to force a timeout of the operation. 
Invoke poll() again so that each
+            // OffsetFetchRequestState is evaluated via isExpired().
+            time.sleep(defaultApiTimeoutMs);
+            commitRequestManager.poll(time.milliseconds());
+            futures.forEach(f -> assertFutureThrows(f, 
TimeoutException.class));
+        } else {
+            futures.forEach(f -> assertFutureThrows(f, KafkaException.class));
+            assertEmptyPendingRequests(commitRequestManager);

Review Comment:
   I wasn't removing the inflight requests since the code in the response 
handler removes it. If we did remove the inflight requests before the response 
was received, the user would see the warning about the request not being in the 
"buffer" when it did complete.



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