Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1990,6 +1990,13 @@ private Fetch<K, V> pollForFetches(Timer timer) {
}
}

// If the background thread has already discovered a metadata error (e.g. TopicAuthorizationException)
// and completed the inflight poll event, skip blocking on the fetch buffer entirely. The error will
// be surfaced by checkInflightPoll on the next iteration of the poll loop. See KAFKA-20397.
if (inflightPoll != null && inflightPoll.error().isPresent()) {
return collectFetch();
}

log.trace("Polling for fetches with timeout {}", pollTimeout);

Timer pollTimer = time.timer(pollTimeout);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@
import org.apache.kafka.common.errors.GroupAuthorizationException;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.InvalidGroupIdException;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.errors.WakeupException;
Expand Down Expand Up @@ -1736,6 +1737,73 @@ public void testPollDoesNotAddNewAsyncPollEventWhenOneIsAlreadyInFlight() {
verify(applicationEventHandler, times(1)).add(isA(AsyncPollEvent.class));
}

/**
* KAFKA-20397: verifies that a metadata error discovered by the background thread between
* {@code checkInflightPoll()} and the blocking wait in {@code pollForFetches()} is surfaced
* promptly, without wasting the full fetch-wait interval.
*
* <p>Scenario: {@code maximumTimeToWait()} returns 100 ms (shorter than the 500 ms poll
* timeout). The background thread completes the inflight {@link AsyncPollEvent} with a
* {@link TopicAuthorizationException} before {@code pollForFetches()} reaches the blocking
* {@code fetchBuffer.awaitWakeup()} call.
*/
@Test
public void testPollSurfacesMetadataErrorWithoutWastingFetchWaitInterval() {
FetchBuffer fetchBuffer = mock(FetchBuffer.class);
ConsumerInterceptors<String, String> interceptors = mock(ConsumerInterceptors.class);
ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = mock(ConsumerRebalanceListenerInvoker.class);
SubscriptionState subscriptions = new SubscriptionState(new LogContext(), AutoOffsetResetStrategy.NONE);
consumer = newConsumer(fetchBuffer, interceptors, rebalanceListenerInvoker, subscriptions);

final String topicName = "topic1";
final TopicPartition tp = new TopicPartition(topicName, 0);

subscriptions.assignFromUser(singleton(tp));
subscriptions.seek(tp, 0);

final long fetchWaitMs = 100L;
doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());

AtomicReference<AsyncPollEvent> capturedEvent = new AtomicReference<>();
doAnswer(invocation -> {
capturedEvent.set(invocation.getArgument(0));
return null;
}).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncPollEvent.class));

AtomicBoolean errorInjected = new AtomicBoolean(false);

// Inject the error inside maximumTimeToWait(), which is the first call in
// pollForFetches — after checkInflightPoll has already submitted the event.
// This models the background thread completing the event with a metadata error
// between checkInflightPoll and the blocking wait.
doAnswer(invocation -> {
if (!errorInjected.get() && capturedEvent.get() != null) {
capturedEvent.get().onMetadataError(
new TopicAuthorizationException(singleton(topicName)));
errorInjected.set(true);
}
return fetchWaitMs;
}).when(applicationEventHandler).maximumTimeToWait();

doAnswer(invocation -> {
Timer pollTimer = invocation.getArgument(0, Timer.class);
((MockTime) time).sleep(pollTimer.remainingMs());
return null;
}).when(fetchBuffer).awaitWakeup(any(Timer.class));

final long pollTimeoutMs = 500;
long startMs = time.milliseconds();

assertThrows(TopicAuthorizationException.class,
() -> consumer.poll(Duration.ofMillis(pollTimeoutMs)));
long elapsedMs = time.milliseconds() - startMs;

assertTrue(elapsedMs < fetchWaitMs,
"Expected error to be surfaced promptly (elapsed " + elapsedMs +
" ms), but consumer wasted a full fetch-wait interval (" + fetchWaitMs + " ms)");
}

/**
* Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer, Predicate) processBackgroundEvents}
* handles the case where the {@link Future} takes a bit of time to complete, but does within the timeout.
Expand Down