lianetm commented on code in PR #17035: URL: https://github.com/apache/kafka/pull/17035#discussion_r1815159364
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ########## @@ -1520,6 +1523,9 @@ private Fetch<K, V> pollForFetches(Timer timer) { return fetch; } + // send any new fetches (won't resend pending fetches) + sendFetches(timer); Review Comment: The actual poll now happens in here (addAndGet that will complete when the background has had one run, called fetchMgr.poll), so should the log line on ln 1538 "Polling for fetches with timeout..." be right before this? ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java: ########## @@ -3386,6 +3394,71 @@ public void testWhenFetchResponseReturnsALeaderShipChangeErrorAndNewLeaderInform assertTrue(subscriptions.isFetchable(tp1)); } + @Test + public void testPollWithoutCreateFetchRequests() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); Review Comment: what do we need these for? This test is about polling the fetchReqMgr without having created requests, which should simply notice it has no pendingFetch and return empty poll result (so I would expect we don't need to setup anything regarding subscriptions) ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java: ########## @@ -65,16 +69,60 @@ protected void maybeThrowAuthFailure(Node node) { networkClientDelegate.maybeThrowAuthFailure(node); } + /** + * Signals the {@link Consumer} wants requests be created for the broker nodes to fetch the next + * batch of records. + * + * @see CreateFetchRequestsEvent + * @return Future on which the caller can wait to ensure that the requests have been created + */ + public CompletableFuture<Void> createFetchRequests() { + CompletableFuture<Void> future = new CompletableFuture<>(); + + if (pendingFetchRequestFuture != null) { + // In this case, we have an outstanding fetch request, so chain the newly created future to be + // completed when the "pending" future is completed. + pendingFetchRequestFuture.whenComplete((value, exception) -> { + if (exception != null) { + future.completeExceptionally(exception); + } else { + future.complete(value); + } + }); + } else { + pendingFetchRequestFuture = future; + } + + return future; + } + /** * {@inheritDoc} */ @Override public PollResult poll(long currentTimeMs) { - return pollInternal( + if (pendingFetchRequestFuture == null) { + // If no explicit request for creating fetch requests was issued, just short-circuit. + return PollResult.EMPTY; + } + + try { + PollResult result = pollInternal( prepareFetchRequests(), this::handleFetchSuccess, this::handleFetchFailure - ); + ); + pendingFetchRequestFuture.complete(null); Review Comment: do we need to complete this future also on `pollOnClose`? there may be a `pendingFetchRequestFuture` there that won't be completed (not that I'm seeing how leaving that future uncompleted on close will cause a problem but seems safer to complete it, consistently with how we do it here after `pollInternal`) ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ########## @@ -707,6 +708,8 @@ public ConsumerRecords<K, V> poll(final Duration timeout) { updateAssignmentMetadataIfNeeded(timer); final Fetch<K, V> fetch = pollForFetches(timer); if (!fetch.isEmpty()) { + sendFetches(timer); Review Comment: at this point we may already have records in hand to return (consumed position updated), so we should be very careful to not throw any error here. But this `sendFetches` could throw interrupted because of the `addAndGet` right? Shouldn't we just do a best effort to pipeline the next requests using `add` instead of `addAndGet`? It would achieve what we want, removing the risk of errors, and it would actually align better with what the classic does on this `sendFetches` + `transmitSends`: https://github.com/apache/kafka/blob/140d35c5459f4c7a91b23a238467cb5aa01b59fb/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java#L327-L329 -- 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