lianetm commented on code in PR #17035:
URL: https://github.com/apache/kafka/pull/17035#discussion_r1773139954
##########
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);
+ return result;
+ } catch (Throwable t) {
+ // A "dummy" poll result is returned here rather than rethrowing
the error because any error
+ // that is thrown from any RequestManager.poll() method interrupts
the polling of the other
+ // request managers.
+ pendingFetchRequestFuture.completeExceptionally(t);
Review Comment:
Even though we propagate the error here, we just ignore it at the consumer
level when doing addAndGet on this result. Should we at least add an error log
for visibility? (there is some usage of the subscription state when handling
the success/failure, so if we get some unexpected error from that, like the no
current assignment, it would go totally silent it seems)
##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java:
##########
@@ -3387,6 +3394,71 @@ public void
testWhenFetchResponseReturnsALeaderShipChangeErrorAndNewLeaderInform
assertTrue(subscriptions.isFetchable(tp1));
}
+ @Test
+ public void testPollWithoutCreateFetchRequests() {
+ buildFetcher();
+
+ assignFromUser(singleton(tp0));
+ subscriptions.seek(tp0, 0);
+
+ assertEquals(0, sendFetches(false));
+ }
+
+ @Test
+ public void testPollWithCreateFetchRequests() {
+ buildFetcher();
+
+ assignFromUser(singleton(tp0));
+ subscriptions.seek(tp0, 0);
+
+ CompletableFuture<Void> future = fetcher.createFetchRequests();
+ assertNotNull(future);
+ assertFalse(future.isDone());
+
+ assertEquals(1, sendFetches(false));
+ assertTrue(future.isDone());
+
+ assertEquals(0, sendFetches(false));
+ }
+
+ @Test
+ public void testPollWithCreateFetchRequestsError() {
+ buildFetcher();
+
+ assignFromUser(singleton(tp0));
+ subscriptions.seek(tp0, 0);
+
+ fetcher.setAuthenticationException(new
AuthenticationException("Intentional error"));
+ CompletableFuture<Void> future = fetcher.createFetchRequests();
+ assertNotNull(future);
+ assertFalse(future.isDone());
+
+ assertDoesNotThrow(() -> sendFetches(false));
+ assertTrue(future.isCompletedExceptionally());
Review Comment:
I guess here we expect the same auth exception? if so we could better
`assertFutureThrows`
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -697,6 +698,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:
this could throw `TimeoutException` I expect (poll with zero/low timeout +
addAndGet) right? before this change, not having enough time to fetch would
just carry on, exit the while(timer.notExpired) and return empty records. I
image we could catch the timeout that the addAndGet could throw, probably log,
and just carry on (to align with the previous behaviour)? (to review on the
other places where we call the sendFEtches too)
--
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]