Re: delivery semantics?

2018-04-05 Thread Jacob Sheck
Your question seems to be confusing the producer and consumer responsibilities in message delivery durability. Neha from Confluent wrote a blog post about this which should help your understanding of the how the Kafka clients handle this.

delivery semantics?

2018-03-30 Thread Victor L
Can someone clarify it: - "At most once": messages may be lost but are never redelivered Does it mean: "commit, when message is received by leader partition, retries disabled for producer no matter of ISR status"? - "At least once": commit only when message is ackd by all partitions in replication

Re: Kafka Streams delivery semantics and state store

2017-02-06 Thread Krzysztof Lesniewski, Nexiot AG
answer. In general I agree with what you are saying. I was however thinking about this one case, in which I think the at-least-once delivery semantics may still be broken. However, I came to conclusion it does not really matter in that application. Here is the case: Let's assume we have

Re: Kafka Streams delivery semantics and state store

2017-02-06 Thread Matthias J. Sax
t;> >>>> >>>> -Matthias >>>> >>>> On 2/2/17 12:55 PM, Eno Thereska wrote: >>>>> Hi Krzysztof, >>>>> >>>>> There are several scenarios where you want a set of records to be >>>>> sent atomi

Re: Kafka Streams delivery semantics and state store

2017-02-06 Thread Krzysztof Lesniewski, Nexiot AG
Thank you Matthias for your answer. In general I agree with what you are saying. I was however thinking about this one case, in which I think the at-least-once delivery semantics may still be broken. However, I came to conclusion it does not really matter in that application. Here is the case

Re: Kafka Streams delivery semantics and state store

2017-02-03 Thread Matthias J. Sax
>>> >>> Thanks >>> Eno >>> >>>> On 2 Feb 2017, at 20:01, Krzysztof Lesniewski, Nexiot AG >>>> <krzysztof.lesniew...@nexiot.ch> wrote: >>>> >>>> Thank you Matthias for your answer. >>>> >>

Re: Kafka Streams delivery semantics and state store

2017-02-03 Thread Krzysztof Lesniewski, Nexiot AG
. I though before about the alternative to idempotent updates you have proposed, but I think it carries a risk of breaking at-least-once delivery semantics in rare cases. Given that state store changelog is flushed first, if an application crashes after flushing the state store, but b

Re: Kafka Streams delivery semantics and state store

2017-02-02 Thread Matthias J. Sax
y to all my cases. >> >> I though before about the alternative to idempotent updates you have >> proposed, but I think it carries a risk of breaking at-least-once delivery >> semantics in rare cases. Given that state store changelog is flushed first, >> if an a

Re: Kafka Streams delivery semantics and state store

2017-02-02 Thread Eno Thereska
as for your answer. > > Of course, wherever it is possible I will use idempotent updates, but > unfortunately it does not apply to all my cases. > > I though before about the alternative to idempotent updates you have > proposed, but I think it carries a risk of breaking at-leas

Re: Kafka Streams delivery semantics and state store

2017-02-02 Thread Krzysztof Lesniewski, Nexiot AG
delivery semantics in rare cases. Given that state store changelog is flushed first, if an application crashes after flushing the state store, but before flushing all the produced records, a reprocessed record may be discarded (considered a duplicate), even though there is a chance its processing

Re: Kafka Streams delivery semantics and state store

2017-02-02 Thread Matthias J. Sax
ld use the record offset as id) -- if the offset in the state is not smaller than the current record's offset, the current record is a duplicate. -Matthias On 2/2/17 6:38 AM, Krzysztof Lesniewski, Nexiot AG wrote: > Hello, > > In multiple sources I read that Kafka Streams has at-least-o

Kafka Streams delivery semantics and state store

2017-02-02 Thread Krzysztof Lesniewski, Nexiot AG
Hello, In multiple sources I read that Kafka Streams has at-least-once delivery semantics, meaning that in case of failure, a message can be processed more than once, but it will not be lost. It is achieved by committing offset only after the message processing is completely finished and all

High Level Consumer delivery semantics

2014-01-29 Thread Clark Breyman
Wrestling through the at-least/most-once semantics of my application and I was hoping for some confirmation of the semantics. I'm not sure I can classify the high level consumer as either type. False ack scenario: - Thread A: call next() on the ConsumerIterator, advancing the PartitionTopicInfo

Re: High Level Consumer delivery semantics

2014-01-29 Thread Guozhang Wang
Hi Clark, In practice, the client app code need to always commit offset after it has processed the messages, and hence only the second case may happen, leading to at least once. Guozhang On Wed, Jan 29, 2014 at 11:51 AM, Clark Breyman cl...@breyman.com wrote: Wrestling through the

Re: High Level Consumer delivery semantics

2014-01-29 Thread Guozhang Wang
Hi Clark, 1. This is true, you need to synchronize these consumer threads when calling commitOffsets(); 2. If you are asking what if the consumer thread crashed after currentTopicInfo.resetConsumeOffset(consumedOffset) within the next() call, then on its startup, it will lose all these