KAFKA-20535: Improve async consumer CPU usage under low max.poll.records.#22199
KAFKA-20535: Improve async consumer CPU usage under low max.poll.records.#22199chickenchickenlove wants to merge 2 commits intoapache:trunkfrom
Conversation
|
Hi, @lianetm ! |
lianetm
left a comment
There was a problem hiding this comment.
Thanks @chickenchickenlove, high level the approach makes sense to me. Just some nits from a first pass.
And some brainstorming on the side, just to make sure there is nothing else that needs attention. Given that you have the repro in hand, can you get some visibility on what part of the background thread is "taking long" to process the inflightPoll that triggers reconciliation? E.g., app thread adds the inflightPoll to the queue and let's assume the member is RECONCILING. From what you describe in the PR, you're seeing that by the time the app thread checks inflightPoll.isReconciliationCheckComplete the background hasn't made it to the point where it sets it to true. So is there anything "delaying" the background process of the inflightPoll that we can improve? (why doesn't it make it to marking reconciliation check done before the app thread needs it? that reconciliation trigger does not block on anything, it just prepares requests if needed and sets some flags in the subscription state). We can address this separately if needed, but raising it here because it's the scenario that makes me wonder why
| * | ||
| * @param memberState The new consumer group member state. | ||
| */ | ||
| default void onConsumerMemberStateChange(MemberState memberState) { |
There was a problem hiding this comment.
should we name it onMemberStateUpdated to be consistent with the existing funcs?
There was a problem hiding this comment.
My original intention was to show that this callback is only needed for the ConsumerGroup. However, since the default method is a no-op, generalizing it as onMemberStateUpdated and letting the call site override it if needed seems like a better approach.
Thank you! 🙇♂️
| private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD); | ||
| private final AtomicInteger refCount = new AtomicInteger(0); | ||
|
|
||
| private volatile boolean memberInReconciliationState = false; |
There was a problem hiding this comment.
the reconciliation state means there is a pending reconciliation. Given here we're at the consumer level (not in the internal state machine impl), would it be clearer to name this around hasPendingReconciliation or similar?
|
@lianetm I don't think this necessarily means that the background thread is slow compared to the application thread. With low On the other hand, adding an The profiling results seem consistent with this interpretation:
So my interpretation is that the extra CPU usage is not from reconciliation work itself being expensive, but from repeatedly going through the wait/future-completion path: the application thread parks while waiting for This patch does not reduce the number of
The main difference is that this patch avoids waiting on For more optimization, while we could consider a further optimization to only enqueue AsyncPollEvent when necessary, given the current profiling results, I believe the resulting performance improvement would be minimal. 🤔 Also, To confirm the assumptions, I can add temporary instrumentation around This should show whether the application thread is reaching What do you think? |
Description
KAFKA-20332 fixed a correctness issue in the async consumer where the application thread could collect buffered records before the background thread had checked for pending reconciliations. The fix added a wait on
inflightPoll.reconciliationCheckFuture()inAsyncKafkaConsumer.collectFetch().This restored the correctness guarantee, but it also increased CPU usage in low
max.poll.recordsscenarios. Withmax.poll.records=5, profiling shows that the additional cost mainly comes from the application thread waiting onConsumerUtils.getResult(inflightPoll.reconciliationCheckFuture(), timeoutMs)even when the consumer group member is not reconciling.This patch avoids that unnecessary wait by tracking the consumer group member state in
AsyncKafkaConsumer.AbstractMembershipManagernow notifiesMemberStateListenerwhenever the consumer member transitionsto a new state.
AsyncKafkaConsumeruses this signal to wait for the reconciliation check only while the member is inMemberState.RECONCILING.Test Condition
bin/kafka-producer-perf-test.shin Broker.before: 5a2dcf8after: 7e1c9dboptimized: this PRTest Result
kubectl top podevery 30second, 10 times. The optimized version reduced CPU usage by
about 23.7% compared with
after.Flame Graph Summary
Aftershows higher CPU usage, and the profile also shows increased time inparkNanosandunpark. This suggests that the additional wait onreconciliationCheckFutureintroduced more application/background thread coordination overhead.AsyncKafkaConsumer.collectFetch
In
after, the application thread spends additional time inConsumerUtils.getResult()while waiting for the reconciliation check future. This also increases related park/unpark activity and application event processing on the background thread. In the optimized version, this wait is skipped unless the member is actually inRECONCILINGstate.
ConsumerNetworkThread.runOnce
The higher CPU usage in
onsumerNetworkThreadappears to be a secondary effect of the application thread waiting onreconciliationCheckFuturemore often. Each wait requires coordination between the applicationthread and the ackground thread: the app thread enqueues an
AsyncPollEvent, waits for the reconciliation check to complete, and the background thread processes that event and completes the future. Asa result,
ConsumerNetworkThread.processApplicationEventsand relatedAsyncPollEventprocessing show higher CPU usage in the after profile.Reviewers: Lianet Magrans lmagrans@confluent.io