-
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
Offsets are reset on Azure EventHubs when a consumer starts #2677
Comments
Only a few times, not as often as on Even Hubs whatsoever , it can be reproduced (perhaps a race condition) with docker using kafka 1.0.0 when having multiple partitions: Configuration topic := "test"
consumerGroup := "bar"
config.Net.TLS.Enable = false
config.Net.SASL.Enable = false
config.Consumer.Offsets.Initial = sarama.OffsetOldest
config.Consumer.Offsets.AutoCommit.Enable = true
config.Consumer.Offsets.AutoCommit.Interval = time.Second
config.Version = sarama.V1_0_0_0
brokerAddresses := []string{"localhost:9092"}
Create topic with multiple partitions:
Publishing a few messages and adding and removing consumers, sometimes, it also resets the offsets and reads all messages again, but not as often as Event Hubs. |
To reproduce it with docker with kafka 1.0.0 there's more chances if we start 3 consumers, publish a few messages, wait around 10 min and stop one of the consumers. Most certainly the offsets will be reset and the other consumers will consume all messages again. |
I've had a dig through the commits between 1.40.1 and 1.41.0. There's no obvious changes to the consumer group logic (or at least I didn't spot any). There are however quite a few changes to support later versions of individual Kafka protocol flows. It looks like #2555 means that setting version Looking at the 1.0.x Kafka client code, it uses a default of -1 if no value is specified: @mfamador - if its not inconvenient, I'd be interested to know whether setting the value of |
@prestona, I guess that's it, setting |
Thanks for the confirmation @mfamador! I'll look at coding up a fix. |
Re-produced using Kafka v1.1.1, and the consumergroups Sarama example: go run main.go -brokers localhost:9092 -topics testtopic -group testgroup -version 1.1.1 It appears that the broker cleans up expired offsets asynchronously, so I have to wait 5-10 minutes for the offsets to be expired. The broker code treats a retention time of -1 as meaning "use the default" here. |
The retention time field of the offset commit request uses -1 to mean "use the broker's default". Sarama uses the value 0 in the `Config.Consumer.Offsets.Retenton` field to mean "use the broker's default". Ensure that Sarama's default (0) is correctly mapped to the broker's default (-1). Fixes: IBM#2677 Signed-off-by: Adrian Preston <[email protected]>
To confirm that I've tested your fix with Azure EvenHubs and it works perfectly! |
The retention time field of the offset commit request uses -1 to mean "use the broker's default". Sarama uses the value 0 in the `Config.Consumer.Offsets.Retenton` field to mean "use the broker's default". Ensure that Sarama's default (0) is correctly mapped to the broker's default (-1). Fixes: #2677 Signed-off-by: Adrian Preston <[email protected]>
Description
Starting from version
1.41.0
, every time a consumer group starts or leaves, all running consumers from the same consumer group consume all messages all over again. Seems like the offsets are being reset and aren't commited again.I couldn't reproduce the issue on a regular kafka, this only occurs apparently using Azure Event Hubs, so kafka version 1.0.0.
Reverting back to 1.40.1 everything works as expected.
Versions
Configuration
Logs
logs: CLICK ME
Additional Context
Not sure if it's related, but on 1.40.1, if we try to use version 2.0.0 on an Event Hub we can not even connect, we need to specify 1.0.0.
Starting from 1.41.0, though, we can set 1.0.0 or 2.0.0 and we're able to connect the same, with the referred problem of reseting the offsets, of course.
The text was updated successfully, but these errors were encountered: