-
Notifications
You must be signed in to change notification settings - Fork 3.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Consumer stalled after commit failure during rebalance #2933
Comments
I forgot to mention, I tested this against librdkafka 1.4.0 as well and the consumer does not stall. Here's the relevant portion of the debug logs.
It looks like the OffsetCommit response is received before the new partitions are assigned, so the |
I just bisected the issue to commit 757b376. |
@edenhill I think I see what's going on here. The above commit changed the semantics of OffsetCommit failures, in particular it retries the OffsetCommit for I can fix the issue by changing the error action for But this fix doesn't sit well with me. Why does an outstanding commit request prevent restarting the partition fetchers in the first place? If there's a good reason, would it make sense to unconditionally restart them when the request succeeds or fails? |
Wow, very impressed by your root cause analysis! 💯 |
This will not make v1.4.4, will address for v1.5.0 |
The reason is that the fetcher might need to resume from the committed offset (which is the default behaviour), so we'll want to make sure any outstanding commits are done before we try to read the commits back from the broker. |
Ok, so we want to delay starting the fetcher until the OffsetCommit request returns to ensure it has an accurate starting point, which could either be (1) the offset we're trying to commit (if the OffsetCommit request succeeds) or (2) the previously-committed offset on the cluster (if the OffsetCommit request fails). In that case, we'd want to unconditionally restart the fetchers when outstanding OffsetCommits return, regardless of success/failure, right? |
…2933) This also refactors the cgrp's OffsetCommit handling.
Thanks for sharing the issue, and nice RCA. Is there any workaround for this, other than using v1.4.0? |
I think it might be possible to call assign() again on the current assignment. |
Ok, let me check that. |
…2933) This also refactors the cgrp's OffsetCommit handling.
Hi! Any plans to fix this? We are seeing this problem in production constantly and this is a huge problem for us. Thanks. |
There's a fix for #2933 in librdkafka v1.5.2 that you'll definitely want to have. |
Besides keeping the worker updated, this release solves a bug, where consumer would randomly get stuck during rebalances. Links: - confluentinc/librdkafka#2933
Besides keeping the worker updated, this release solves a bug, where consumer would randomly get stuck during rebalances. Links: - confluentinc/librdkafka#2933
Can't reproduce on master |
This mitigates confluentinc/librdkafka#2933 which is fixed in 1.5.2. This issue was leading to hanging consumers when they tried to commit during a rebalance operation. Signed-off-by: Matthias Wahl <[email protected]>
This mitigates confluentinc/librdkafka#2933 which is fixed in 1.5.2. This issue was leading to hanging consumers when they tried to commit during a rebalance operation. Signed-off-by: Matthias Wahl <[email protected]>
This mitigates confluentinc/librdkafka#2933 which is fixed in 1.5.2. This issue was leading to hanging consumers when they tried to commit during a rebalance operation. Signed-off-by: Matthias Wahl <[email protected]> Signed-off-by: Heinz N. Gies <[email protected]>
Read the FAQ first: https://github.com/edenhill/librdkafka/wiki/FAQ
Description
If an OffsetCommit request overlaps with a rebalance, the partition fetcher threads are not restarted until the OffsetCommit response is received. If the commit fails, they are not restarted at all, and the consumer appears to hang.
How to reproduce
See https://gist.github.com/gridaphobe/d1c544631c9569af810b405e572144cd. We force a commit failure by unassigning ourselves before committing, which causes the commit to be processed by the broker after the generation id has been incremented. (The actual consumer code where we discovered this issue does an asynchronous commit on a regular interval, which only sometimes overlaps with rebalances in this way.) Start one instance of the consumer, and after it has received a few messages start another one. After the rebalance, only the new consumer will receive messages even though the partitions have been evenly distributed. The original consumer never recovers, even if you shut down the other one.
IMPORTANT: Always try to reproduce the issue on the latest released version (see https://github.com/edenhill/librdkafka/releases), if it can't be reproduced on the latest version the issue has been fixed.
Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
1.4.2
2.2.0
debug=consumer,cgrp
) from librdkafka: https://gist.github.com/gridaphobe/d1c544631c9569af810b405e572144cd#file-consumer-1-log and https://gist.github.com/gridaphobe/d1c544631c9569af810b405e572144cd#file-consumer-2-logThe text was updated successfully, but these errors were encountered: