diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java
index eec41c6d3b4f1..cba2b65cbba7d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java
@@ -20,6 +20,7 @@
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult;
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
+import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent;
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
import org.apache.kafka.clients.consumer.internals.metrics.HeartbeatMetricsManager;
@@ -241,7 +242,7 @@ public PollResult pollOnClose(long currentTimeMs) {
* are sent, so blocking for longer than the heartbeat interval might mean the application thread is not
* responsive to changes.
*
- *
Similarly, we may have to unblock the application thread to send a `PollApplicationEvent` to make sure
+ *
Similarly, we may have to unblock the application thread to send a {@link AsyncPollEvent} to make sure
* our poll timer will not expire while we are polling.
*
*
In the event that heartbeats are currently being skipped, this still returns the next heartbeat
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
index 3915ff7c8df88..e806b21c46558 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
@@ -41,6 +41,7 @@
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent;
import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent;
+import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent;
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
import org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent;
@@ -59,7 +60,6 @@
import org.apache.kafka.clients.consumer.internals.events.LeaveGroupOnCloseEvent;
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent;
import org.apache.kafka.clients.consumer.internals.events.PausePartitionsEvent;
-import org.apache.kafka.clients.consumer.internals.events.PollEvent;
import org.apache.kafka.clients.consumer.internals.events.ResetOffsetEvent;
import org.apache.kafka.clients.consumer.internals.events.ResumePartitionsEvent;
import org.apache.kafka.clients.consumer.internals.events.SeekUnvalidatedEvent;
@@ -325,8 +325,7 @@ private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() {
// Init value is needed to avoid NPE in case of exception raised in the constructor
private Optional clientTelemetryReporter = Optional.empty();
- // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates
- private boolean cachedSubscriptionHasAllFetchPositions;
+ private AsyncPollEvent inflightPoll;
private final WakeupTrigger wakeupTrigger = new WakeupTrigger();
private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker;
@@ -464,7 +463,8 @@ public AsyncKafkaConsumer(final ConsumerConfig config,
final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext,
metadata,
subscriptions,
- requestManagersSupplier);
+ requestManagersSupplier
+ );
this.applicationEventHandler = applicationEventHandlerFactory.build(
logContext,
time,
@@ -623,7 +623,7 @@ public AsyncKafkaConsumer(final ConsumerConfig config,
new RebalanceCallbackMetricsManager(metrics)
);
ApiVersions apiVersions = new ApiVersions();
- Supplier networkClientDelegateSupplier = () -> new NetworkClientDelegate(
+ Supplier networkClientDelegateSupplier = NetworkClientDelegate.supplier(
time,
config,
logContext,
@@ -834,23 +834,19 @@ public ConsumerRecords poll(final Duration timeout) {
throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions");
}
- do {
- PollEvent event = new PollEvent(timer.currentTimeMs());
- // Make sure to let the background thread know that we are still polling.
- // This will trigger async auto-commits of consumed positions when hitting
- // the interval time or reconciling new assignments
- applicationEventHandler.add(event);
- // Wait for reconciliation and auto-commit to be triggered, to ensure all commit requests
- // retrieve the positions to commit before proceeding with fetching new records
- ConsumerUtils.getResult(event.reconcileAndAutoCommit(), defaultApiTimeoutMs.toMillis());
+ // This distinguishes the first pass of the inner do/while loop from subsequent passes for the
+ // inflight poll event logic.
+ boolean firstPass = true;
+ do {
// We must not allow wake-ups between polling for fetches and returning the records.
// If the polled fetches are not empty the consumed position has already been updated in the polling
// of the fetches. A wakeup between returned fetches and returning records would lead to never
// returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches.
wakeupTrigger.maybeTriggerWakeup();
- updateAssignmentMetadataIfNeeded(timer);
+ checkInflightPoll(timer, firstPass);
+ firstPass = false;
final Fetch fetch = pollForFetches(timer);
if (!fetch.isEmpty()) {
// before returning the fetched records, we can send off the next round of fetches
@@ -878,6 +874,107 @@ public ConsumerRecords poll(final Duration timeout) {
}
}
+ /**
+ * {@code checkInflightPoll()} manages the lifetime of the {@link AsyncPollEvent} processing. If it is
+ * called when no event is currently processing, it will start a new event processing asynchronously. A check
+ * is made during each invocation to see if the inflight event has completed. If it has, it will be
+ * processed accordingly.
+ */
+ private void checkInflightPoll(Timer timer, boolean firstPass) {
+ if (firstPass && inflightPoll != null) {
+ // Handle the case where there's a remaining inflight poll from the *previous* invocation
+ // of AsyncKafkaConsumer.poll().
+ maybeClearPreviousInflightPoll();
+ }
+
+ boolean newlySubmittedEvent = false;
+
+ if (inflightPoll == null) {
+ inflightPoll = new AsyncPollEvent(calculateDeadlineMs(timer), time.milliseconds());
+ newlySubmittedEvent = true;
+ log.trace("Inflight event {} submitted", inflightPoll);
+ applicationEventHandler.add(inflightPoll);
+ }
+
+ try {
+ // Note: this is calling user-supplied code, so make sure that any errors thrown here are caught and
+ // the inflight event is cleared.
+ offsetCommitCallbackInvoker.executeCallbacks();
+ processBackgroundEvents();
+ } catch (Throwable t) {
+ // If an exception was thrown during execution of offset commit callbacks or background events,
+ // bubble it up to the user but make sure to clear out the inflight request because the error effectively
+ // renders it complete.
+ log.trace("Inflight event {} failed due to {}, clearing", inflightPoll, String.valueOf(t));
+ inflightPoll = null;
+ throw ConsumerUtils.maybeWrapAsKafkaException(t);
+ } finally {
+ timer.update();
+ }
+
+ if (inflightPoll != null) {
+ maybeClearCurrentInflightPoll(newlySubmittedEvent);
+ }
+ }
+
+ private void maybeClearPreviousInflightPoll() {
+ if (inflightPoll.isComplete()) {
+ Optional errorOpt = inflightPoll.error();
+
+ if (errorOpt.isPresent()) {
+ // If the previous inflight event is complete, check if it resulted in an error. If there was
+ // an error, throw it without delay.
+ KafkaException error = errorOpt.get();
+ log.trace("Previous inflight event {} completed with an error ({}), clearing", inflightPoll, error);
+ inflightPoll = null;
+ throw error;
+ } else {
+ // Successful case...
+ if (fetchBuffer.isEmpty()) {
+ // If it completed without error, but without populating the fetch buffer, clear the event
+ // so that a new event will be enqueued below.
+ log.trace("Previous inflight event {} completed without filling the buffer, clearing", inflightPoll);
+ inflightPoll = null;
+ } else {
+ // However, if the event completed, and it populated the buffer, *don't* create a new event.
+ // This is to prevent an edge case of starvation when poll() is called with a timeout of 0.
+ // If a new event was created on *every* poll, each time the event would have to complete the
+ // validate positions stage before the data in the fetch buffer is used. Because there is
+ // no blocking, and effectively a 0 wait, the data in the fetch buffer is continuously ignored
+ // leading to no data ever being returned from poll().
+ log.trace("Previous inflight event {} completed and filled the buffer, not clearing", inflightPoll);
+ }
+ }
+ } else if (inflightPoll.isExpired(time) && inflightPoll.isValidatePositionsComplete()) {
+ // The inflight event validated positions, but it has expired.
+ log.trace("Previous inflight event {} expired without completing, clearing", inflightPoll);
+ inflightPoll = null;
+ }
+ }
+
+ private void maybeClearCurrentInflightPoll(boolean newlySubmittedEvent) {
+ if (inflightPoll.isComplete()) {
+ Optional errorOpt = inflightPoll.error();
+
+ if (errorOpt.isPresent()) {
+ // If the inflight event completed with an error, throw it without delay.
+ KafkaException error = errorOpt.get();
+ log.trace("Inflight event {} completed with an error ({}), clearing", inflightPoll, error);
+ inflightPoll = null;
+ throw error;
+ } else {
+ log.trace("Inflight event {} completed without error, clearing", inflightPoll);
+ inflightPoll = null;
+ }
+ } else if (!newlySubmittedEvent) {
+ if (inflightPoll.isExpired(time) && inflightPoll.isValidatePositionsComplete()) {
+ // The inflight event validated positions, but it has expired.
+ log.trace("Inflight event {} expired without completing, clearing", inflightPoll);
+ inflightPoll = null;
+ }
+ }
+ }
+
/**
* Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and
* partitions.
@@ -1773,16 +1870,27 @@ private Fetch pollForFetches(Timer timer) {
return fetch;
}
- // send any new fetches (won't resend pending fetches)
- sendFetches(timer);
-
- // We do not want to be stuck blocking in poll if we are missing some positions
- // since the offset lookup may be backing off after a failure
+ // With the non-blocking poll design, it's possible that at this point the background thread is
+ // concurrently working to update positions. Therefore, a _copy_ of the current assignment is retrieved
+ // and iterated looking for any partitions with invalid positions. This is done to avoid being stuck
+ // in poll for an unnecessarily long amount of time if we are missing some positions since the offset
+ // lookup may be backing off after a failure.
+ if (pollTimeout > retryBackoffMs) {
+ Set partitions = subscriptions.assignedPartitions();
- // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we MUST call
- // updateAssignmentMetadataIfNeeded before this method.
- if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > retryBackoffMs) {
- pollTimeout = retryBackoffMs;
+ if (partitions.isEmpty()) {
+ // If there aren't any assigned partitions, this could mean that this consumer's group membership
+ // has not been established or assignments have been removed and not yet reassigned. In either case,
+ // reduce the poll time for the fetch buffer wait.
+ pollTimeout = retryBackoffMs;
+ } else {
+ for (TopicPartition tp : partitions) {
+ if (!subscriptions.hasValidPosition(tp)) {
+ pollTimeout = retryBackoffMs;
+ break;
+ }
+ }
+ }
}
log.trace("Polling for fetches with timeout {}", pollTimeout);
@@ -1811,19 +1919,19 @@ private Fetch pollForFetches(Timer timer) {
* of the {@link #fetchBuffer}, converting it to a well-formed {@link CompletedFetch}, validating that it and
* the internal {@link SubscriptionState state} are correct, and then converting it all into a {@link Fetch}
* for returning.
- *
- *
- *
- * This method will {@link ConsumerNetworkThread#wakeup() wake up the network thread} before returning. This is
- * done as an optimization so that the next round of data can be pre-fetched.
*/
private Fetch collectFetch() {
- final Fetch fetch = fetchCollector.collectFetch(fetchBuffer);
-
- // Notify the network thread to wake up and start the next round of fetching.
- applicationEventHandler.wakeupNetworkThread();
+ // With the non-blocking async poll, it's critical that the application thread wait until the background
+ // thread has completed the stage of validating positions. This prevents a race condition where both
+ // threads may attempt to update the SubscriptionState.position() for a given partition. So if the background
+ // thread has not completed that stage for the inflight event, don't attempt to collect data from the fetch
+ // buffer. If the inflight event was nulled out by checkInflightPoll(), that implies that it is safe to
+ // attempt to collect data from the fetch buffer.
+ if (inflightPoll != null && !inflightPoll.isValidatePositionsComplete()) {
+ return Fetch.empty();
+ }
- return fetch;
+ return fetchCollector.collectFetch(fetchBuffer);
}
/**
@@ -1836,11 +1944,10 @@ private Fetch collectFetch() {
* defined
*/
private boolean updateFetchPositions(final Timer timer) {
- cachedSubscriptionHasAllFetchPositions = false;
try {
CheckAndUpdatePositionsEvent checkAndUpdatePositionsEvent = new CheckAndUpdatePositionsEvent(calculateDeadlineMs(timer));
wakeupTrigger.setActiveTask(checkAndUpdatePositionsEvent.future());
- cachedSubscriptionHasAllFetchPositions = applicationEventHandler.addAndGet(checkAndUpdatePositionsEvent);
+ applicationEventHandler.addAndGet(checkAndUpdatePositionsEvent);
} catch (TimeoutException e) {
return false;
} finally {
@@ -1858,41 +1965,6 @@ private boolean isCommittedOffsetsManagementEnabled() {
return groupMetadata.get().isPresent();
}
- /**
- * This method signals the background thread to {@link CreateFetchRequestsEvent create fetch requests}.
- *
- *
- *
- * This method takes the following steps to maintain compatibility with the {@link ClassicKafkaConsumer} method
- * of the same name:
- *
- *
- * -
- * The method will wait for confirmation of the request creation before continuing.
- *
- * -
- * The method will throw exceptions encountered during request creation to the user immediately.
- *
- * -
- * The method will suppress {@link TimeoutException}s that occur while waiting for the confirmation.
- * Timeouts during request creation are a byproduct of this consumer's thread communication mechanisms.
- * That exception type isn't thrown in the request creation step of the {@link ClassicKafkaConsumer}.
- * Additionally, timeouts will not impact the logic of {@link #pollForFetches(Timer) blocking requests}
- * as it can handle requests that are created after the timeout.
- *
- *
- *
- * @param timer Timer used to bound how long the consumer waits for the requests to be created, which in practice
- * is used to avoid using {@link Long#MAX_VALUE} to wait "forever"
- */
- private void sendFetches(Timer timer) {
- try {
- applicationEventHandler.addAndGet(new CreateFetchRequestsEvent(calculateDeadlineMs(timer)));
- } catch (TimeoutException swallow) {
- // Can be ignored, per above comments.
- }
- }
-
/**
* This method signals the background thread to {@link CreateFetchRequestsEvent create fetch requests} for the
* pre-fetch case, i.e. right before {@link #poll(Duration)} exits. In the pre-fetch case, the application thread
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
index d2d178a88c38b..67656cf327b5b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
@@ -20,9 +20,9 @@
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
-import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.CompletableEvent;
import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper;
+import org.apache.kafka.clients.consumer.internals.events.MetadataErrorNotifiableEvent;
import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics;
import org.apache.kafka.common.internals.IdempotentCloser;
import org.apache.kafka.common.requests.AbstractRequest;
@@ -40,6 +40,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Objects;
+import java.util.Optional;
import java.util.concurrent.BlockingQueue;
import java.util.function.Supplier;
@@ -193,10 +194,13 @@ private void processApplicationEvents() {
try {
if (event instanceof CompletableEvent) {
applicationEventReaper.add((CompletableEvent>) event);
- // Check if there are any metadata errors and fail the CompletableEvent if an error is present.
- // This call is meant to handle "immediately completed events" which may not enter the awaiting state,
- // so metadata errors need to be checked and handled right away.
- maybeFailOnMetadataError(List.of((CompletableEvent>) event));
+ }
+ // Check if there are any metadata errors and fail the event if an error is present.
+ // This call is meant to handle "immediately completed events" which may not enter the
+ // awaiting state, so metadata errors need to be checked and handled right away.
+ if (event instanceof MetadataErrorNotifiableEvent) {
+ if (maybeFailOnMetadataError(List.of(event)))
+ continue;
}
applicationEventProcessor.process(event);
} catch (Throwable t) {
@@ -368,18 +372,26 @@ void cleanup() {
/**
* If there is a metadata error, complete all uncompleted events that require subscription metadata.
*/
- private void maybeFailOnMetadataError(List> events) {
- List> subscriptionMetadataEvent = new ArrayList<>();
+ private boolean maybeFailOnMetadataError(List> events) {
+ List filteredEvents = new ArrayList<>();
- for (CompletableEvent> ce : events) {
- if (ce instanceof CompletableApplicationEvent && ((CompletableApplicationEvent>) ce).requireSubscriptionMetadata())
- subscriptionMetadataEvent.add((CompletableApplicationEvent>) ce);
+ for (Object obj : events) {
+ if (obj instanceof MetadataErrorNotifiableEvent) {
+ filteredEvents.add((MetadataErrorNotifiableEvent) obj);
+ }
}
- if (subscriptionMetadataEvent.isEmpty())
- return;
- networkClientDelegate.getAndClearMetadataError().ifPresent(metadataError ->
- subscriptionMetadataEvent.forEach(event -> event.future().completeExceptionally(metadataError))
- );
+ // Don't get-and-clear the metadata error if there are no events that will be notified.
+ if (filteredEvents.isEmpty())
+ return false;
+
+ Optional metadataError = networkClientDelegate.getAndClearMetadataError();
+
+ if (metadataError.isPresent()) {
+ filteredEvents.forEach(e -> e.onMetadataError(metadataError.get()));
+ return true;
+ } else {
+ return false;
+ }
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java
index c52b5453e21d9..e7139657d5abf 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchRequestManager.java
@@ -145,6 +145,14 @@ private PollResult pollInternal(FetchRequestPreparer fetchRequestPreparer,
try {
Map fetchRequests = fetchRequestPreparer.prepare();
+ if (fetchRequests.isEmpty()) {
+ // If there's nothing to fetch, wake up the FetchBuffer so it doesn't needlessly wait for a wakeup
+ // that won't come until the data in the fetch buffer is consumed.
+ fetchBuffer.wakeup();
+ pendingFetchRequestFuture.complete(null);
+ return PollResult.EMPTY;
+ }
+
List requests = fetchRequests.entrySet().stream().map(entry -> {
final Node fetchTarget = entry.getKey();
final FetchSessionHandler.FetchRequestData data = entry.getValue();
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java
index e85f244c80472..762718b2035d8 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java
@@ -471,4 +471,33 @@ protected NetworkClientDelegate create() {
}
};
}
+
+ /**
+ * Creates a {@link Supplier} for deferred creation during invocation by
+ * {@link ConsumerNetworkThread}.
+ */
+ public static Supplier supplier(final Time time,
+ final ConsumerConfig config,
+ final LogContext logContext,
+ final KafkaClient client,
+ final Metadata metadata,
+ final BackgroundEventHandler backgroundEventHandler,
+ final boolean notifyMetadataErrorsViaErrorQueue,
+ final AsyncConsumerMetrics asyncConsumerMetrics) {
+ return new CachedSupplier<>() {
+ @Override
+ protected NetworkClientDelegate create() {
+ return new NetworkClientDelegate(
+ time,
+ config,
+ logContext,
+ client,
+ metadata,
+ backgroundEventHandler,
+ notifyMetadataErrorsViaErrorQueue,
+ asyncConsumerMetrics
+ );
+ }
+ };
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java
index 501821a231028..4f41886e9ce4f 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java
@@ -38,13 +38,13 @@
import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper;
import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
import org.apache.kafka.clients.consumer.internals.events.EventProcessor;
-import org.apache.kafka.clients.consumer.internals.events.PollEvent;
import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgeAsyncEvent;
import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgeOnCloseEvent;
import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgeSyncEvent;
import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgementCommitCallbackEvent;
import org.apache.kafka.clients.consumer.internals.events.ShareAcknowledgementCommitCallbackRegistrationEvent;
import org.apache.kafka.clients.consumer.internals.events.ShareFetchEvent;
+import org.apache.kafka.clients.consumer.internals.events.SharePollEvent;
import org.apache.kafka.clients.consumer.internals.events.ShareSubscriptionChangeEvent;
import org.apache.kafka.clients.consumer.internals.events.ShareUnsubscribeEvent;
import org.apache.kafka.clients.consumer.internals.events.StopFindCoordinatorOnCloseEvent;
@@ -385,7 +385,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) {
backgroundEventQueue, time, asyncConsumerMetrics);
final Supplier networkClientDelegateSupplier =
- () -> new NetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics);
+ NetworkClientDelegate.supplier(time, config, logContext, client, metadata, backgroundEventHandler, true, asyncConsumerMetrics);
GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
config,
@@ -586,7 +586,7 @@ public synchronized ConsumerRecords poll(final Duration timeout) {
do {
// Make sure the network thread can tell the application is actively polling
- applicationEventHandler.add(new PollEvent(timer.currentTimeMs()));
+ applicationEventHandler.add(new SharePollEvent(timer.currentTimeMs()));
processBackgroundEvents();
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java
index 0441566462a0c..eee77b7ae9a09 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java
@@ -18,6 +18,7 @@
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.internals.events.AsyncPollEvent;
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
import org.apache.kafka.clients.consumer.internals.metrics.HeartbeatMetricsManager;
@@ -426,7 +427,7 @@ public StreamsMembershipManager membershipManager() {
* are sent, so blocking for longer than the heartbeat interval might mean the application thread is not
* responsive to changes.
*
- * Similarly, we may have to unblock the application thread to send a `PollApplicationEvent` to make sure
+ *
Similarly, we may have to unblock the application thread to send a {@link AsyncPollEvent} to make sure
* our poll timer will not expire while we are polling.
*
*
In the event that heartbeats are currently being skipped, this still returns the next heartbeat
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java
index cb23e6aaf2826..2db2b16b1d268 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java
@@ -21,14 +21,14 @@
import java.util.List;
import java.util.Map;
-public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent