librdkafka
librdkafka copied to clipboard
Consumers should not be able to commit during a rebalance
Description
Now that librdkafka supports cooperative sticky partitions assignment strategy, we should ensure that consumers that commit offsets manually can’t commit offsets during rebalance as it triggers a follow up rebalance.
I don’t think there is any valid use cases for allowing this type of behavior:
- This behavior could lead to an infinite amount of rebalances. Rebalances will only stop once none of the consumers attempt to commit offsets during a rebalance.
- The Java Kafka Consumer lib does not appear to allow clients to commit offsets during a rebalance
- Auto commit callback in librdkafka does not commit offsets when there is an ongoing rebalance
I believe this issue with manual auto commit is isolated to cooperative sticky strategy because when a consumer uses the eager strategy, a rebalance starts with all the partitions being revoked and ends when new partitions have been assigned to the consumer. As a result, the consumer will never attempt to commit offsets because there are no offsets to be committed. Of course, if the consumer uses cooperative sticky, we can’t ensure that the consumer won’t attempt to commit offsets during a rebalance as the consumer might still own partitions during a rebalance. Furthermore, clients have no way of knowing that a rebalance is ongoing or not and so they can’t prevent consumers from committing offsets when necessary.
I see three potential solutions to this problem, but I think the first one makes the most sense to implement:
- Add the same check that exists in the auto commit callback in rd_kafka_commit. We can throw an error to the client to let them know why we did not attempt to commit offsets
- Provide a new endpoint to librdkafka clients that allows them to check whether a rebalance is ongoing or not, so they are able to prevent consumers from committing offsets on their end
- Provided the option to client (by passing a flag of some sort) to decide whether they want
rd_kafka_committo not attempt to commit offsets during rebalances
Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
- [x] librdkafka version (release number or git tag):
v1.8.2 - [x] Apache Kafka version:
v2.7.1 - [x] librdkafka client configuration:
"partition.assignment.strategy": "cooperative-sticky"
"enable.auto.commit": false,
- [ ] Operating system:
<REPLACE with e.g., Centos 5 (x64)> - [ ] Provide logs (with
debug=..as necessary) from librdkafka - [ ] Provide broker log excerpts
- [ ] Critical issue
I think the root of problem lies in OffsetCommit response error handling, we're probably getting a ERR_ILLEGAL_GENERATION back from the coordinator at this point, which causes us to rejoin the group - thus the rebalance. Even if we add a rebalancing check to commit() it is possible for us to receive this error anyway, due to timing, so I think it might be better to also improve the error handling here and not trigger a rejoin if the group is rebalancing.
Agreed, improving the error handling would make a great difference as well. I haven't been able to confirm this yet, but I believe a new rebalance is also triggered when the error is ERR_REBALANCE_IN_PROGRESS
It would be great if this behaviour were documented somewhere in the wiki or in the config description. We spent some efforts to understand what the heck is going on and why the cooperative-sticky rebalance is not working as expected and leads to endless rebalances.
We use confluent_kafka for python. I tried to implement a workaround using on_assign/on_revoke callbacks and prevent manual commits for some time when these callbacks are triggered. But noticed the group generation id can be changed even before these callbacks are triggered. As I understand currently with confluent_kafka python there is now way to know for sure the rebalance is in progress and we can't use manual commits with cooperative-sticky. Also I noticed when auto commit enabled librdkafka can send OffsetCommitRequests when rebalance is in progress and it doesn't lead to invalid group generation id error. I see this offset commit happens between the on_revoke callback triggered in one consumer and on_assign in another.
so I think it might be better to also improve the error handling here and not trigger a rejoin if the group is rebalancing.
Is not this contradicts to the behavior described in this KIP document? It says:
If received UNKNOWN_MEMBER_ID or ILLEGAL_GENERATION from join-group / sync-group / commit / heartbeat response: reset generation / clear member-id correspondingly, call rebalance listener's onPartitionsLost for all the partition and then re-join group with empty assigned partition.
Just adding an additional data point that we see this exact behavior with our setup in kafka. I can try to take a crack at fixing this in my off time, but wondering if there is anyone on Confluent side working on this @milindl @emasab since if you're using manual offset commits, you cannot use Cooperative rebalancing today in any context.
is there any ongoing work for this?
@scanterog see #4220 , but in short, I think that this issue has dropped off the radar. Unfortunately the fix that @roxelo created (and I attempted to get merged) broke other behavior in librdkafka. @milindl said the librdkafka team was looking into this internally, but unfortunately from my side there are few things I can do as my place of work doesn't pay for confluent support and therefore we have no sway on their roadmap. If anyone who is interested in getting this more attention on the confluent side and is a paying confluent customer I highly recommend they push via their TAMs to get traction on this.