Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Conversation

kirktrue
Copy link
Contributor

@kirktrue kirktrue commented Sep 10, 2025

Introduces AsyncPollEvent to make the poll event handling in
AsyncKafkaConsumer and ApplicationEventProcessor non-blocking to avoid
performance bottlenecks. The new approach enables multi-stage polling
logic, where possible.

… max poll value

Introduces CompositePollEvent and CompositePollResult to refactor and streamline the poll event handling in AsyncKafkaConsumer and ApplicationEventProcessor. The new approach enables multi-step polling logic, improves callback and background event processing, and enhances testability. Also adds size methods to BackgroundEventHandler and OffsetCommitCallbackInvoker, disables several tests, and updates related classes to support the new event flow.
@github-actions github-actions bot added triage PRs from the community consumer clients labels Sep 10, 2025
Added a Javadoc comment to the RequiresApplicationThreadExecution interface to clarify its purpose and usage, specifically regarding the need to interrupt CompositePollEvent processing when requiresApplicationThread() returns true.
…or to check metadata errors

Refactors AsyncKafkaConsumer, ShareConsumerImpl, and ApplicationEventProcessor to inject NetworkClientDelegate using a supplier method. Adds a static supplier factory to NetworkClientDelegate for deferred instantiation. Updates related tests and construction logic to support the new dependency injection approach.
Copy link

A label of 'needs-attention' was automatically added to this PR in order to raise the
attention of the committers. Once this issue has been triaged, the triage label
should be removed to prevent this automation from happening again.

Replaces CompletableFuture-based state handling in CompositePollEvent with a new Blocker class for improved synchronization and exception handling. Updates AsyncKafkaConsumer, WakeupTrigger, ApplicationEventProcessor, and related tests to use Blocker, simplifying event completion and error propagation.
…KA-18376-chain-events-in-background-thread-without-addAndGet
Introduces an AtomicBoolean to track completion state in CompositePollEvent and updates ApplicationEventProcessor to mark events as complete when appropriate. Refactors AsyncKafkaConsumer to use a new CompositePollEventInvoker for polling, replacing prepareFetch, and implements exponential backoff for incomplete events.
Refactored AsyncKafkaConsumer and related classes to improve composite poll event handling, including explicit state management and pausing for background event processing or offset commit callbacks. Metadata errors are now optionally propagated via a dedicated error field in NetworkClientDelegate, allowing for more flexible error handling. Updated tests and logging to reflect these changes.
Adds NetworkClientDelegate as a dependency to ApplicationEventProcessor and updates AsyncKafkaConsumer and ShareConsumerImpl to supply it. Introduces error handling in composite poll processing using metadata errors from NetworkClientDelegate. Updates related tests to mock the new dependency.
Eliminated the BackgroundEventHandler parameter from OffsetsRequestManager and its usages in RequestManagers and related tests. This simplifies the constructor and removes unnecessary dependencies.
@github-actions github-actions bot removed needs-attention triage PRs from the community labels Sep 25, 2025
Eliminated an unused import of KafkaException from ConsumerPollTestUtils.java to clean up the code.
Refactors the composite poll event flow to use explicit state transitions and result handling, replacing Optional-based APIs with more direct methods. Introduces clearer state management for CompositePollEvent, updates ApplicationEventProcessor and CompositePollEventProcessorContext to use new completion and error handling methods, and improves documentation and test utilities. Updates tests and utility methods to match the new APIs and behaviors.
Replaces nextEventType with startingEventType in CompositePollEvent and related classes for improved clarity and correctness. Adds validation for allowed starting event types, updates method names, improves logging, and enhances documentation for event processing context and state transitions.
Surfaced isPendingCallbacks() at the SubscriptionState API so that partitions that are pending callback invocation are not returned. This is also in agreement with the JavaDoc which states:

"This will give the set of topic partitions currently assigned to the consumer (which may be none if . . . the partitions are in the process of getting reassigned)."
@kirktrue kirktrue changed the title [WIP] KAFKA-18376: High CPU load when AsyncKafkaConsumer uses a small max poll value KAFKA-18376: High CPU load when AsyncKafkaConsumer uses a small max poll value Sep 30, 2025
Updated comments in CompositePollEvent for clarity and improved the debug log message in CompositePollEventInvoker to better describe event failure handling.
…on thread callbacks

FetchBuffer is now provided to CompositePollEventProcessorContext and its supplier, allowing the context to call fetchBuffer.wakeup() after completing an event. The wakeup method in FetchBuffer is made public to support this usage.
…group offsets tests

Refactored tests to use a counter for consumed records and added explicit consumer configuration with AUTO_OFFSET_RESET set to 'earliest'. This ensures all produced records are consumed and improves reliability of partition assignment checks.
Copy link
Member

@lianetm lianetm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the updates!


private void process(final PollEvent event) {
processPollEvent(event.pollTimeMs());
event.markReconcileAndAutoCommitComplete();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would expect this is not needed anymore, and actually doesn't seem used. This used to be to ensure we didn't move onto generating a fetch before retrieving positions to commit (both steps were independently triggered from the app thread), but that's not the case anymore right?)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd noticed that the ShareConsumerImpl was still using PollEvents, so I didn't want to muck with it too much. It seems like I could refactor processPollEvent() into two separate pieces of logic, one for the CompositePollEvent and one for the (Share consumer) PollEvent.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since ShareConsumerImpl doesn't use the Future from the event, I'll remove it and clean that up.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

Comment on lines 253 to 260
if (nextEventType == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) {
log.debug("Processing {} logic for {}", nextEventType, event);
processUpdatePatternSubscriptionEvent();
nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS;

if (maybeFailCompositePoll(event) || maybePauseCompositePoll(event, nextEventType))
return;
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see this hasn't been addressed. Seems like unneeded states/transitions that make the CompositePollEvent more complex, but I could be missing why we may need it like this?

Comment on lines 764 to 765
if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType))
return;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this logic is to maybe jump back to the app thread, but at this point we haven't really done anything in the background yet, so couldn't we avoid this hop and check for this in the app thread directly?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybeCompleteExceptionally() checks NetworkClientDelegate.getAndClearMetadataError(). We can't access that from the application thread, so this is the first available opportunity to do so.

It's possible for there to be offset commits that finished after the last AsyncPollEvent completed, so maybeCompleteWithCallbackRequired() catches those cases before we've started any new work on the new poll.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybeCompleteExceptionally() checks NetworkClientDelegate.getAndClearMetadataError(). We can't access that from the application thread, so this is the first available opportunity to do so.

I was thinking an atomic variable. Metadata errors are discovered in the background indeed, and we need to read it from the app thread on poll (and clear it I guess). Maybe an atomic var shared via applicationEventHandler, for instance? (ApplicationEventHandler lives in the AsyncConsumer, and has ref to the network component, seems like the component in-between). The network layer sets the atomic var, we read it on poll and throw if there is anything there. Would that work? If it works, we could solve the inter-thread communication for this case with it, instead of jumping back and forth between the threads.

It's possible for there to be offset commits that finished after the last AsyncPollEvent completed, so maybeCompleteWithCallbackRequired() catches those cases before we've started any new work on the new poll.

Agree that we may have commits that completed after the last AsyncPollEvent, but why do we need to go to the background to trigger them? I was imagining if we could just trigger commit callbacks in the app thread before triggering the (next) AsyncPollEvent

Comment on lines 257 to 258
// all commit request generation points have been passed,
// so it's safe to notify the app thread could proceed and start fetching
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this should be removed (referred to the markReconcileAndAutoCommitComplete that was removed)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

processPollEvent(event.pollTimeMs());
nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA;

if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Totally agree with maybeCompleteWithCallbackRequired here (right after triggering a reconciliation, we may have callbacks to run)

But I'm still not convinced about the need for maybeCompleteExceptionally here. That one is to propagate metadata errors we may have received in MetadataReponses. The classic consumer would throw them on the first call to client.poll hit from the consumer.poll (could be dealing with the coordinator, or later on). With the async consumer, I would imagine it should be enough to have a single point within the consumer.poll where we "maybeThrowMetadataErrors"

We already have an initial check to "maybeThrowMetadataErrors" (where the comment is to maybe move to the app thread even). Shouldn't that be enough?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't believe we can introduce a maybeThrowMetadataErrors step in the application thread as it needs to access the NetworkClientDelegate owned by the background thread, plus it would need to be a blocking call.

I will experiment with leaving the first call to maybeCompleteExceptionally() in the process() method, and removing the subsequent checks.

Comment on lines 90 to 103
} else if (state == CompositePollEvent.State.CALLBACKS_REQUIRED) {
// The background thread detected that it needed to yield to the application thread to invoke
// callbacks. Even though the inflight reference _should_ be overwritten when the next stage of
// the event is submitted, go ahead and clear out the inflight request just to be sure.
log.trace("Event {} paused for callbacks, clearing inflight", inflight);
inflight = null;

// Note: this is calling user-supplied code, so make sure to handle possible errors.
applicationThreadCallbacks.run();

// The application thread callbacks are complete. Create another event to resume the polling at
// the next stage.
submitEvent(result.asNextEventType(), timer);
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't quite get why we need this whole new state to trigger rebalance/commit callbacks?

We already have thread-safe queues where elems are added IF there are callbacks that need to be executed, so I was expecting that we simply need to check if there is something on those queues and run it at the beginning of each poll iteration?

    public void poll(Timer timer) {

        // trigger rebalance and commit callbacks if any - what applicationThreadCallbacks.run() does
        offsetCommitCallbackInvoker.executeCallbacks();
        processBackgroundEvents();

        if (inflight == null) {
            log.trace("No existing inflight event, submitting a new event");
            submitEvent(ApplicationEvent.Type.POLL, timer);
        }
        ....

Would that work?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the current implementation, the commit/rebalance callbacks are executed on the application thread after the PollEvent and before the CheckAndUpdatePositionsEvent. The CALLBACKS_REQUIRED state in this implementation is used to exit from the background thread at that in-between point to effectively achieve the same timing result.

I've made a few attempts to move the callbacks to execute before the events and remove the state, as it is clunky. However, doing so fundamentally changes the established timing. This is further reflected by the fact that multiple unit/integration tests start to fail when the timing is changed by removing the state.

});
}

nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this whole flow is already looking much simpler, nice! but still, this intermediate state seems to be only to ensure we don't attempt to trigger reconciliations/callbacks/metadataErrors on the first poll after the one where one of those were triggered, correct? But do we really need that? It would mean that we would skip triggering a reconciliation when it's maybe needed (would happen on a following poll, but why skipping it in the current one?)

Alternatively, I imagine we could simply trigger all actions on each poll (trigger reconciliation, commit, updateFetchPositions, trigger fetch), short-circuiting as needed back to app thread, but no need to carry on from where we left of given that is all about triggering actions that will simply be no-op if repeated and not needed (and if we end up triggering reconciliation/callbacks/errors on 2 subsequent polls, it's because it's indeed needed)

All those steps have their own "flow control" btw (maybeReconcile will only do something if no other reconciliation going on, metadata errors are cleared when thrown, auto-commit considers the interval and inflights...) that's why I wonder if we can just simply trigger them all on each iteration, no more state machine on the ASYNC_POLL?

Comment on lines 764 to 765
if (context.maybeCompleteExceptionally(event) || context.maybeCompleteWithCallbackRequired(event, nextEventType))
return;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybeCompleteExceptionally() checks NetworkClientDelegate.getAndClearMetadataError(). We can't access that from the application thread, so this is the first available opportunity to do so.

I was thinking an atomic variable. Metadata errors are discovered in the background indeed, and we need to read it from the app thread on poll (and clear it I guess). Maybe an atomic var shared via applicationEventHandler, for instance? (ApplicationEventHandler lives in the AsyncConsumer, and has ref to the network component, seems like the component in-between). The network layer sets the atomic var, we read it on poll and throw if there is anything there. Would that work? If it works, we could solve the inter-thread communication for this case with it, instead of jumping back and forth between the threads.

It's possible for there to be offset commits that finished after the last AsyncPollEvent completed, so maybeCompleteWithCallbackRequired() catches those cases before we've started any new work on the new poll.

Agree that we may have commits that completed after the last AsyncPollEvent, but why do we need to go to the background to trigger them? I was imagining if we could just trigger commit callbacks in the app thread before triggering the (next) AsyncPollEvent

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants