-
Notifications
You must be signed in to change notification settings - Fork 1.8k
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
Re-ordering of offset commit requests can cause committed offset to move "backwards". #2940
Comments
prestona
added a commit
to prestona/sarama
that referenced
this issue
Jul 19, 2024
Add locking to prevent concurrent calls to commit from potentially being re-ordered between determining which offsets to include in the offset commit request, and sending the request to Kafka. Fixes: IBM#2940 Co-authored-by: Michael Burgess <michburg@uk.ibm.com> Signed-off-by: Adrian Preston <PRESTONA@uk.ibm.com>
prestona
added a commit
to prestona/sarama
that referenced
this issue
Jul 19, 2024
Add locking to prevent concurrent calls to commit from potentially being re-ordered between determining which offsets to include in the offset commit request, and sending the request to Kafka. Move finding the coordinator before creating the request to avoid holding the lock in the case the coordinator is unknown. This requires a change to the "new offset manager" test, which previously didn't expect the mock broker to receive a find coordinator request. Fixes: IBM#2940 Co-authored-by: Michael Burgess <michburg@uk.ibm.com> Signed-off-by: Adrian Preston <PRESTONA@uk.ibm.com>
prestona
added a commit
to prestona/sarama
that referenced
this issue
Jul 23, 2024
Hold broker lock to prevent concurrent calls to commit from potentially being re-ordered between determining which offsets to include in the offset commit request, and sending the request to Kafka. Move finding the coordinator before creating the request to avoid holding the lock in the case the coordinator is unknown. This requires a change to the "new offset manager" test, which previously didn't expect the mock broker to receive a find coordinator request. Fixes: IBM#2940 Co-authored-by: Michael Burgess <michburg@uk.ibm.com> Signed-off-by: Adrian Preston <PRESTONA@uk.ibm.com>
I've opened a couple of PR's for this one.
I think I favor #2941, but either should fix this issue. |
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Description
Offsets committed by multiple consumer group sessions can be re-ordered by the time they arrive at Kafka, potentially resulting in duplicate message delivery, should the offset be used to resume consumption.
If a single sarama.ConsumerGroup is consuming from more than one partition, then reordering of offset commit requests can occur.
This is because of a race condition between the goroutines used to run the ConsumerClaim method for different topic partitions.
When ConsumerGroupSession.Commit() is called, this ends up calling into offsetManager.flushToBroker().
This in turn:
However, there is no locking to prevent the interleaving of two (or more) go routines between the building an offset commit request, and sending the request to Kafka.
For example, consider the following interactions between two goroutines P0 and P1 - each belonging to the same consumer group, but consuming from different partitions of a topic:
From the Kafka broker's perspective, it receives:
This causes the offset committed for P0 to move "backwards" from 12 to 11.
Due to the nature of this race condition, it is more likely to occur if:
Versions
Configuration
Logs
Not applicable. Sarama does not emit any logging relating to this problem.
Additional Context
The text was updated successfully, but these errors were encountered: