Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
169 commits
Select commit Hold shift + click to select a range
34932e2
[WIP] KAFKA-18376: High CPU load when AsyncKafkaConsumer uses a small…
kirktrue Sep 10, 2025
b5d7d01
[WIP] More work on correctness
kirktrue Sep 11, 2025
d4802c7
Re-enabling tests in AsyncKafkaConsumer
kirktrue Sep 11, 2025
9fd7e58
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Sep 15, 2025
d3fa910
Minor clean up from design review
kirktrue Sep 15, 2025
dbc4773
Updates to fix inverted logic in maybeInterruptCompositePoll()
kirktrue Sep 15, 2025
09f8cb5
Add documentation for RequiresApplicationThreadExecution
kirktrue Sep 15, 2025
5e794ce
Inject NetworkClientDelegate via supplier for ApplicationEventProcess…
kirktrue Sep 15, 2025
464d5ba
Removed the verbose logging
kirktrue Sep 16, 2025
d253b84
Work in progress to get past most of the integration test issues
kirktrue Sep 17, 2025
aaefbef
Clean up logic related to metadata errors that can happen along any s…
kirktrue Sep 17, 2025
40f6754
Minor updates for CompletableEventReaper logging
kirktrue Sep 17, 2025
3e0b920
Refactor CompositePollEvent to use Blocker for state management
kirktrue Sep 18, 2025
529aab3
Update AsyncKafkaConsumer.java
kirktrue Sep 18, 2025
784aad2
Moving toward a non-blocking poll() implementation
kirktrue Sep 19, 2025
c6a7923
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Sep 19, 2025
524782c
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thre…
kirktrue Sep 19, 2025
00f9069
Merge branch 'KAFKA-18376-chain-events-in-background-thread' into KAF…
kirktrue Sep 19, 2025
0ac19f9
Clean up
kirktrue Sep 19, 2025
ae0ddcc
Add completion tracking to CompositePollEvent
kirktrue Sep 20, 2025
6775aac
Refactor poll event handling and metadata error propagation
kirktrue Sep 20, 2025
2c3547e
Inject NetworkClientDelegate into ApplicationEventProcessor
kirktrue Sep 20, 2025
18f4fa1
Remove BackgroundEventHandler from OffsetsRequestManager
kirktrue Sep 20, 2025
ea99a13
Handle immediate metadata errors for CompletableEvents
kirktrue Sep 20, 2025
56062f5
Update NetworkClientDelegate.java
kirktrue Sep 20, 2025
9d65fa2
Merge pull request #10 from kirktrue/KAFKA-18376-chain-events-in-back…
kirktrue Sep 20, 2025
99304db
Remove extra whitespace in NetworkClientDelegate
kirktrue Sep 20, 2025
702b257
Revert removal of contains() from CompletableEventReaper
kirktrue Sep 20, 2025
81b707e
Update NetworkClientDelegate.java
kirktrue Sep 20, 2025
8159884
Refactor application thread requirement handling
kirktrue Sep 20, 2025
7112022
Update AsyncKafkaConsumer.java
kirktrue Sep 20, 2025
f40a4ac
Refactor consumer record polling in tests
kirktrue Sep 23, 2025
abaa4dc
Reset backoff on event submission in AsyncKafkaConsumer
kirktrue Sep 23, 2025
1570f69
Handle exceptions in AsyncKafkaConsumer poll event
kirktrue Sep 23, 2025
1e52282
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Sep 23, 2025
2d21fa0
Refactor poll event handling and metadata error management
kirktrue Sep 23, 2025
b193770
Refactor CompositePollEventInvoker to standalone class
kirktrue Sep 23, 2025
2302427
Remove debug logging from CompositePollEventInvoker
kirktrue Sep 24, 2025
91e881f
Fix typo in CompositePollPseudoEvent class name
kirktrue Sep 24, 2025
8b33f08
Refactor lambda in waitForConsumerPollException call
kirktrue Sep 24, 2025
2aaca8d
Move pollForRecords helper method in KafkaConsumerTest
kirktrue Sep 24, 2025
1f1ae24
Remove commented-out event processing code
kirktrue Sep 24, 2025
985bbd7
Improve consumer poll reliability in integration tests
kirktrue Sep 24, 2025
5af8fc4
Updates and fixes for a couple of integration tests
kirktrue Sep 29, 2025
52c0845
Refactoring and clean up
kirktrue Sep 29, 2025
bfcd7ec
More clean up and refactoring
kirktrue Sep 29, 2025
f45b70e
Refactoring
kirktrue Sep 29, 2025
8235ed2
Refactoring and clean up
kirktrue Sep 29, 2025
a157071
Refactor and cleanup
kirktrue Sep 29, 2025
164dfdd
Remove unused KafkaException import in ConsumerPollTestUtils
kirktrue Sep 29, 2025
72472c5
Refactor composite poll event processing and context
kirktrue Sep 29, 2025
5c99d81
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Sep 30, 2025
0aed4af
Minor refactoring and added documentation
kirktrue Sep 30, 2025
ab42fca
Fixed streams tests in PlaintextAdminIntegrationTest
kirktrue Sep 30, 2025
cc49db8
Update CompositePollEvent.java
kirktrue Sep 30, 2025
7ba6047
Improve logging and comments in poll event classes
kirktrue Sep 30, 2025
bc660d6
Reverting changes made to omit pending-callback partitions from being…
kirktrue Sep 30, 2025
e9dbc61
Notify FetchBuffer from background thread if pausing due to applicati…
kirktrue Oct 1, 2025
5d1a34d
Relying on reading records to determine group membership for streams …
kirktrue Oct 1, 2025
5041a36
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 1, 2025
f6864a3
Refactor POLL into ASYNC_POLL and SHARE_POLL
kirktrue Oct 2, 2025
0256bdf
Moving the regex subscription check out of the poll path
kirktrue Oct 2, 2025
a233e90
Remove unused processUpdatePatternSubscriptionEvent method
kirktrue Oct 3, 2025
2b2f70c
Using Consumer.poll() helper methods from TestUtils
kirktrue Oct 3, 2025
461ffdd
Change debug to trace in ApplicationEventProcessor
kirktrue Oct 3, 2025
9fb9ee9
Refactoring
kirktrue Oct 3, 2025
a8ccdb6
Replace pollRecordsUntilTrue with waitUntilTrue in verifyConsumerWith…
kirktrue Oct 4, 2025
e1cf7b7
Replace awaitNonEmptyRecords with waitForRecords in PlaintextConsumer…
kirktrue Oct 4, 2025
7ca4cc8
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 6, 2025
eace3ee
Add detailed Javadoc to MetadataErrorNotifiable
kirktrue Oct 6, 2025
767316b
Refactoring to remove interim callback step
kirktrue Oct 6, 2025
f4eb161
Inline trackCheckAndUpdatePositionsForTimeout in process(AsyncPollEvent)
kirktrue Oct 6, 2025
ddf0423
Revert wakeup() method visibility back to package-private
kirktrue Oct 6, 2025
dcbe761
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 6, 2025
b5f4c86
Minor refactoring, cleanup, extraneous whitespace removal
kirktrue Oct 6, 2025
cac80ef
Reverted minor import change with SaslClientsWithInvalidCredentialsTest
kirktrue Oct 6, 2025
fa784e5
Refactored AsyncPollEvent to remove State
kirktrue Oct 6, 2025
5be7210
Update ApplicationEventProcessor.java
kirktrue Oct 6, 2025
19ce01a
Removed check-and-update-positions future timeout and removed AsyncPo…
kirktrue Oct 6, 2025
b04b0a2
Removed superfluous imports
kirktrue Oct 6, 2025
d939675
Added comment to NetworkClientDelegate.doSend() change
kirktrue Oct 6, 2025
91409bc
Make ConsumerIntegrationTest.testRackAwareAssignment() poll more freq…
kirktrue Oct 6, 2025
e7b5386
Fix stupid typo for poll timeout
kirktrue Oct 7, 2025
4985c7d
Updates for MetadataErrorNotifiableEvent
kirktrue Oct 7, 2025
2e5a982
Revert change to AsyncKafkaConsumer
kirktrue Oct 7, 2025
68f5bc1
Whitespace change
kirktrue Oct 7, 2025
5215030
Update AsyncKafkaConsumer.java
kirktrue Oct 7, 2025
f660b15
Refactor regex subscription evaluation
kirktrue Oct 7, 2025
dfdd693
Removed logging from process(AsyncPollEvent) path
kirktrue Oct 8, 2025
626bb67
Fixed variable name typo
kirktrue Oct 8, 2025
ef5e4da
Added brief documentation for the inner OnSubscriptionUpdatedCallback…
kirktrue Oct 8, 2025
3595e82
Renamed OnSubscriptionUpdatedCallback -> MembershipManagerShim
kirktrue Oct 8, 2025
4e9c313
Revert unnecessary changes to ConsumerIntegrationTest
kirktrue Oct 8, 2025
f9c0345
Minor correction in code comment
kirktrue Oct 8, 2025
293f08b
ConsumerIntegrationTest needs more time to resolve the assignment
kirktrue Oct 8, 2025
f26e832
Reworking logic in pollForFetches
kirktrue Oct 8, 2025
dfd3e7f
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 8, 2025
ba0fdac
Removed an outdated comment in ApplicationEventProcessor
kirktrue Oct 8, 2025
c664ddf
Revert change for onConsumerPoll() to limit to the async consumer and…
kirktrue Oct 8, 2025
45af571
Revert some changes in KafkaConsumerTest
kirktrue Oct 8, 2025
328cb13
Minor tweaks to KafkaConsumerTest to minimize diff noise
kirktrue Oct 8, 2025
5109b82
Revert changes to PlaintextConsumerTest
kirktrue Oct 8, 2025
0bde790
Update PlaintextConsumerTest.java
kirktrue Oct 8, 2025
0e1180f
Reverting changes to PlaintextConsumerCommitTest
kirktrue Oct 8, 2025
1a4fe5d
Update PlaintextConsumerCommitTest.java
kirktrue Oct 8, 2025
802d7d1
Revert more changes to AsyncKafkaConsumerTest
kirktrue Oct 8, 2025
9a673fb
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 8, 2025
cd516e3
Fixed code tweak
kirktrue Oct 8, 2025
089bb24
Test clean up
kirktrue Oct 9, 2025
8e86cf4
Removed whitespace diffs
kirktrue Oct 9, 2025
22b4806
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 9, 2025
8ccf306
Merge branch 'apache:trunk' into KAFKA-18376-chain-events-in-backgrou…
kirktrue Oct 9, 2025
2baf96e
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 9, 2025
9833010
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 9, 2025
36dae3d
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 9, 2025
ef07ee1
Trigger build
kirktrue Oct 10, 2025
afed9fc
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 10, 2025
1bf56bf
Reverted NetworkClientDelegate authentication error check
kirktrue Oct 10, 2025
7df60c7
Refactored completeExceptionally to set the error _first_, then set t…
kirktrue Oct 10, 2025
390ac04
Update AsyncKafkaConsumer.java
kirktrue Oct 11, 2025
7eaa225
Merge branch 'apache:trunk' into KAFKA-18376-chain-events-in-backgrou…
kirktrue Oct 11, 2025
1775d73
Merge branch 'apache:trunk' into KAFKA-18376-chain-events-in-backgrou…
kirktrue Oct 12, 2025
4772675
Remove group.id configuration from testConsumeUsingAssignWithNoAccess
kirktrue Oct 12, 2025
eb77ad8
Added whitespace to README.md to force rebuild
kirktrue Oct 13, 2025
51a04b8
Removing whitespace to kick off build
kirktrue Oct 14, 2025
09d7d0b
Merge branch 'apache:trunk' into KAFKA-18376-chain-events-in-backgrou…
kirktrue Oct 14, 2025
632abf4
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 15, 2025
61bca4c
Restoring ConsumerIntegrationTest timeout and fixing pre-assignment t…
kirktrue Oct 15, 2025
b5f6620
Merge branch 'apache:trunk' into KAFKA-18376-chain-events-in-backgrou…
kirktrue Oct 16, 2025
e3e35bc
Changed incorrect references to PollApplicationEvent to AsyncPollEvent
kirktrue Oct 17, 2025
2ece70e
Merge branch 'apache:trunk' into KAFKA-18376-chain-events-in-backgrou…
kirktrue Oct 17, 2025
8e37061
Reverted change in PlaintextConsumerTest
kirktrue Oct 17, 2025
5d49fda
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 17, 2025
ae89075
Removed unnecessary import
kirktrue Oct 17, 2025
9629d52
Update KafkaConsumerTest.testCurrentLag() to move the poll() to insid…
kirktrue Oct 18, 2025
ed5b163
Re-added call to poll() in testCurrentLag()
kirktrue Oct 18, 2025
ac1b651
Update KafkaConsumerTest.java
kirktrue Oct 18, 2025
d90f326
Merge branch 'apache:trunk' into KAFKA-18376-chain-events-in-backgrou…
kirktrue Oct 19, 2025
998b16e
Merge branch 'apache:trunk' into KAFKA-18376-chain-events-in-backgrou…
kirktrue Oct 20, 2025
421c1c1
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 22, 2025
3870023
Removing unused size() method call from BackgroundEventHandler and Of…
kirktrue Oct 22, 2025
3588239
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 22, 2025
965570b
Handle previous inflight poll events in AsyncKafkaConsumer
kirktrue Oct 23, 2025
4818eb1
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 24, 2025
a1889f9
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 24, 2025
cdaa914
Fix race condition in async poll event position validation
kirktrue Oct 25, 2025
3ec3dd1
Suppress CyclomaticComplexity in checkInflightPoll :(
kirktrue Oct 25, 2025
f07bb28
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 25, 2025
8beb0a8
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 27, 2025
a549a76
Wake up FetchBuffer when no fetch requests are sent to avoid unnecess…
kirktrue Oct 27, 2025
121f6af
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 27, 2025
b884773
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 27, 2025
258bbfb
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 28, 2025
ed554ca
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 28, 2025
ddfbca3
Updated expiration logic for inflight poll events
kirktrue Oct 28, 2025
2fbe4f8
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 29, 2025
315d304
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 29, 2025
1b8c8ef
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 29, 2025
b697d0f
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 30, 2025
09c3306
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 30, 2025
91871d3
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 30, 2025
61da9e4
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 31, 2025
5946e7b
Refactor checkInflightPoll to be less complex
kirktrue Oct 31, 2025
0907013
Update tests and migrate some from AsyncKafkaConsumerTest to Applicat…
kirktrue Oct 31, 2025
5084985
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Oct 31, 2025
0c808e9
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread
kirktrue Nov 3, 2025
a2aa8b4
Fixed comments
kirktrue Nov 3, 2025
ca3ccdb
Verify when matchesSubscriptionPattern is called
kirktrue Nov 3, 2025
7ef75a1
Updating the Timer after running any callbacks and/or background events
kirktrue Nov 3, 2025
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 @@ -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;
Expand Down Expand Up @@ -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.
*
* <p>Similarly, we may have to unblock the application thread to send a `PollApplicationEvent` to make sure
* <p>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.
*
* <p>In the event that heartbeats are currently being skipped, this still returns the next heartbeat
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;

Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -368,18 +372,26 @@ void cleanup() {
/**
* If there is a metadata error, complete all uncompleted events that require subscription metadata.
*/
private void maybeFailOnMetadataError(List<CompletableEvent<?>> events) {
List<CompletableApplicationEvent<?>> subscriptionMetadataEvent = new ArrayList<>();
private boolean maybeFailOnMetadataError(List<?> events) {
List<MetadataErrorNotifiableEvent> 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<Exception> metadataError = networkClientDelegate.getAndClearMetadataError();

if (metadataError.isPresent()) {
filteredEvents.forEach(e -> e.onMetadataError(metadataError.get()));
return true;
} else {
return false;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,14 @@ private PollResult pollInternal(FetchRequestPreparer fetchRequestPreparer,
try {
Map<Node, FetchSessionHandler.FetchRequestData> 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<UnsentRequest> requests = fetchRequests.entrySet().stream().map(entry -> {
final Node fetchTarget = entry.getKey();
final FetchSessionHandler.FetchRequestData data = entry.getValue();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -471,4 +471,33 @@ protected NetworkClientDelegate create() {
}
};
}

/**
* Creates a {@link Supplier} for deferred creation during invocation by
* {@link ConsumerNetworkThread}.
*/
public static Supplier<NetworkClientDelegate> 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
);
}
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -385,7 +385,7 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) {
backgroundEventQueue, time, asyncConsumerMetrics);

final Supplier<NetworkClientDelegate> 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,
Expand Down Expand Up @@ -586,7 +586,7 @@ public synchronized ConsumerRecords<K, V> 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();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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.
*
* <p>Similarly, we may have to unblock the application thread to send a `PollApplicationEvent` to make sure
* <p>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.
*
* <p>In the event that heartbeats are currently being skipped, this still returns the next heartbeat
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,14 @@
import java.util.List;
import java.util.Map;

public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent<Map<String, List<PartitionInfo>>> {
public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent<Map<String, List<PartitionInfo>>> implements MetadataErrorNotifiableEvent {

protected AbstractTopicMetadataEvent(final Type type, final long deadlineMs) {
super(type, deadlineMs);
}

@Override
public boolean requireSubscriptionMetadata() {
return true;
public void onMetadataError(Exception metadataError) {
future().completeExceptionally(metadataError);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,14 @@
public abstract class ApplicationEvent {

public enum Type {
COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE,
COMMIT_ASYNC, COMMIT_SYNC, ASYNC_POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE,
LIST_OFFSETS, CHECK_AND_UPDATE_POSITIONS, RESET_OFFSET, TOPIC_METADATA, ALL_TOPICS_METADATA,
TOPIC_SUBSCRIPTION_CHANGE, TOPIC_PATTERN_SUBSCRIPTION_CHANGE, TOPIC_RE2J_PATTERN_SUBSCRIPTION_CHANGE,
UPDATE_SUBSCRIPTION_METADATA, UNSUBSCRIBE,
CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED,
COMMIT_ON_CLOSE, CREATE_FETCH_REQUESTS, LEAVE_GROUP_ON_CLOSE, STOP_FIND_COORDINATOR_ON_CLOSE,
PAUSE_PARTITIONS, RESUME_PARTITIONS, CURRENT_LAG,
SHARE_FETCH, SHARE_ACKNOWLEDGE_ASYNC, SHARE_ACKNOWLEDGE_SYNC,
SHARE_POLL, SHARE_FETCH, SHARE_ACKNOWLEDGE_ASYNC, SHARE_ACKNOWLEDGE_SYNC,
SHARE_SUBSCRIPTION_CHANGE, SHARE_UNSUBSCRIBE,
SHARE_ACKNOWLEDGE_ON_CLOSE,
SHARE_ACKNOWLEDGEMENT_COMMIT_CALLBACK_REGISTRATION,
Expand Down
Loading