Re: Kafka Streams delivery semantics and state store

2017-02-06 Thread Krzysztof Lesniewski, Nexiot AG
Ok, so now I understand it. Thank you for all the clarifications :) Kind Regards Krzysztof Lesniewski On 06.02.2017 18:36, Matthias J. Sax wrote: Completely agree. Very good and deep analysis! -Matthias On 2/6/17 3:15 AM, Krzysztof Lesniewski, Nexiot AG wrote: Thank you Matthias for your ans

Re: Kafka Streams delivery semantics and state store

2017-02-06 Thread Matthias J. Sax
Completely agree. Very good and deep analysis! -Matthias On 2/6/17 3:15 AM, Krzysztof Lesniewski, Nexiot AG wrote: > 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 > se

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
Answers inline. -Matthias On 2/3/17 7:37 AM, Krzysztof Lesniewski, Nexiot AG wrote: > Thank you Eno for the information on KIP-98. Making downstream topic and > state store's changelog writes atomic would do simplify the problem. I > did not dive into the design, so I am not able to tell if it w

Re: Kafka Streams delivery semantics and state store

2017-02-03 Thread Krzysztof Lesniewski, Nexiot AG
Thank you Eno for the information on KIP-98. Making downstream topic and state store's changelog writes atomic would do simplify the problem. I did not dive into the design, so I am not able to tell if it would bring other implications, but as KIP-98 is so far under discussion, I have to settle

Re: Kafka Streams delivery semantics and state store

2017-02-02 Thread Matthias J. Sax
You assumptions is not completely correct. After a crash and State Store restore, the store will contain exactly the same data as written to the underlying changelog. Thus, if you update was buffered but never send, the store will not contain the update after restore and thus the record will not b

Re: Kafka Streams delivery semantics and state store

2017-02-02 Thread Eno Thereska
Hi Krzysztof, There are several scenarios where you want a set of records to be sent atomically (to a statestore, downstream topics etc). In case of failure then, either all of them commit successfully, or none does. We are working to add exactly-once processing to Kafka Streams and I suspect y

Re: Kafka Streams delivery semantics and state store

2017-02-02 Thread Krzysztof Lesniewski, Nexiot AG
Thank you Matthias 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-least-once de

Re: Kafka Streams delivery semantics and state store

2017-02-02 Thread Matthias J. Sax
Hi, About message acks: writes will be acked, however async (not sync as you describe it). Only before an actual commit, KafkaProducer#flush is called and all not-yet received acks are collected (ie, blocking/sync) before the commit is done. About state guarantees: there are none -- state might b

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