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
KafkaConsumer and current positions #118
Comments
There is a librdkafka issue for this: While it makes sense to commit when reaching EOF even if there were no messages, it is effectively the same as setting auto.offset.reset=earliest. @ewencp Thoughts? As you say position() will return the application's current offsets, but if there are no messages on a partition there is no offset to return, do you think it should return the current partition high-watermark offset? |
Why is that effectively the same? Having auto.offset.reset=earliest and your offset commit expires, it seems like you'd go back and reprocess data? Actually the behavior described for |
And if you have |
There is a difference between where it tries to fetch (pre fetch position) and the application's actual position. |
There are edge cases though. Here is an (edge-case) example of how messages can be lost if a job has an unfortunate downtime (however short):
Because offsets were expired despite the job was running and attempting to re-commit its offsets previously, this job will lose messages. The job may go down because of re-deploying for a bug fix, etc, you just need to be lucky enough to listen to a not busy topic and then have a downtime when offsets are expired. Typically in prod-ready environment when all the data matters you wouldn't use This is why things like I think that allowing jobs to re-assure their offsets would be logical in many reasons. |
Thanks for the thorough explanation, Alexey. This boils down to two things:
I would be happy to add a config property for the EOF one (as not to change the default behaviour), but I'm a little bit hesitant on the latter. |
I was not talking about auto commits (since we don't use them) but about explicit commit calls: when I call I am surprised that this behaviour is considered to be a bug in kafka clients... Could you please explain to me why this behaviour is undesirable? |
This is one of those corner-cases where you'll just need to make a design decision, both options have pros and cons. If you have a use-case for re-committing offsets I'd be happy to add that as an option to librdkafka (can't change the default though). |
My point here is simple: messages frequency should not impact reasonability. I could compensate with much longer offsets expiration settings, but then it impacts the broker performance AFAIK. |
This needs to be fixed in librdkafka, created upstream issue here: confluentinc/librdkafka#1372 |
Closing this issue in favour of the librdkafka one. The librdkafka fix will not require changes to the Python code. |
When the consumer starts and there are no new messages for the group it is impossible to know current positions and to commit current offsets.
Committing offsets even if there are no new messages can be important because offsets in Kafka can expire. To prevent offsets from expiring we need to commit them periodically, even is there are no changes.
Normal
commit()
doesn't help because it won't do anything and would returnKafkaError.NO_OFFSETS
.It is possible to use
commit(offsets)
and give it specific offsets, but how to get them?Currently even if the consumer gets partitions assigned,
consumer.assignment()
returns an empty list when there are no new messages. I strongly suspect that even if I receive messages it will only give me partitions for the received messages, not all of them.OK, I can get a list of assigned partitions from
on_assign
callback. But then when I callconsumer.position(_assigned)
it won't give me current offsets for the group, only for the partitions I received messages from.Adding the ability to query currently assigned offsets would be very helpful.
Making
commit()
committing offsets even if nothing changed would be even more helpful.The text was updated successfully, but these errors were encountered: