Carl,
I double if the change you proposed will have at-least-once guarantee.
consumedOffset
is the next offset of the message that is being returned from
iterator.next(). For example the message returned is A with offset 1 and
then consumedOffset will be 2 set to currentTopicInfo. While the consum
OK, thanks. I agree, the current code is better if you get lots of
rebalancing, and you can do your own thing for stronger guarantees.
For the new consumer, it looks like it should be possible to use multiple
threads, as long as partition order is preserved in the processing, right?
So, one can bu
Yes, your approach works. I am not sure if we should take this as default
solution, though. User can have a simple wrapper + customized rebalance
listener. The tricky part is that the rebalance listener might need
different implementations. So it looks the current API provides enough
simplicity and
Thanks Jiangjie
So you agree that with the modified ConsumerIterator.next() code, the high
level consumer becomes at-least-once, even with auto-commit enabled? That
is what I really want to know.
I'll have a look at the rebalancing code. I think I understand: during
rebalancing, with auto-commit
Hi Carl,
Generally, you approach works to guarantee at least once consumption -
basically people have to commit offset only after they have processed the
message.
The only problem is that in old high level consumer, during consumer
rebalance consumer will (and should) commit offsets. To guarantee
Thanks Bhavesh.
I understand that to get "exactly once" processing of a message requires
some de-duplication. What I'm saying, is that the current high level
consumer, with automatic offset commits enabled, gives neither "at most
once" nor "at least once" guarantees: A consumer group might get dup
HI Carl,
Produce side retry can produce duplicated message being sent to brokers
with different offset with same message. Also, you may get duplicated when
the High Level Consumer offset is not being saved or commit but you have
processed data and your server restart etc...
To guaranteed at-lea
Hi Carl,
"** Disclaimer: I know there's a new consumer API on the way, this mail is
about the currently available API. I also apologise if the below has
already been discussed previously. I did try to check previous discussions
It seems to me that the high-level consumer would be able to support
a
So Carl Heymann's ConsumerIterator.next hack approach is not reasonable?
2015-06-17 08:12:50 + 上のメッセージ Stevo Slavić:
--047d7bfcf30ed09b460518b241db
Content-Type: text/plain; charset=UTF-8
With auto-commit one can only have at-most-once delivery guarantee - after
commit but before messa
With auto-commit one can only have at-most-once delivery guarantee - after
commit but before message is delivered for processing, or even after it is
delivered but before it is processed, things can fail, causing event not to
be processed, which is basically same outcome as if it was not delivered.
Hi
** Disclaimer: I know there's a new consumer API on the way, this mail is
about the currently available API. I also apologise if the below has
already been discussed previously. I did try to check previous discussions
on ConsumerIterator **
It seems to me that the high-level consumer would be
11 matches
Mail list logo