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 be between X and Z
(that's what "at-least-once" can provide -- not more). Thus, it's
recommended to apply idempotent updates to the stores if possible.

As an alternative, you might also be able to add the latest "update
record" to the state itself to depulicate in failure case. For example,
instead of putting <K,V> in the state, you put <K<V,Id>> with id being a
record id that did the last modification on V. Thus, each time before
you update the state, you can check if you did already "add" the current
record to the state (for example, you could 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-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
> intermediate state/data is flushed to a reliable storage. There are
> however some details I want to verify, especially related to Kafka
> Stream's State Store.
> 
> Let's take as an example a Kafka Streams application, which consumes
> messages from topic A, processes the messages using state store S and
> outputs the processing results to topic B. Ensuring at-least-once
> delivery semantics would mean following order:
> 
> 1. Message is processed
> 2. Updated state of state store S (update caused by applying processed
>    message) is sent to backing changelog topic *and acknowledged*
> 3. Processing result is sent to topic B *and acknowledged*
> 4. Offset of topic A is commited (may be deferred)
> 
> Acknowledging is necessary, as otherwise offset could be committed, even
> though messages or state were not successfully submitted, e.g. because
> of batching or because there are less available replicas than
> /min.insync.replicas/ and messages are being buffered. Am I correct here?
> 
> Then, regarding state store, what state will I end up with after a
> failure? Let's assume following scenario:
> 
> 1. I consume message K, and process it in context of state X.
>    Processing the message resulted in changing the state from X to Y.
> 2. I consume the next message, message L, and process it in context of
>    state Y. Processing the message resulted in changing the state from
>    Y to Z.
> 3. Server crashed. As no offset was committed, I will start processing
>    again from message K.
> 
> What state will I end up with when processing again the message K. Are
> there any guaranties that the state store will give me state X, or will
> it be any state between X and Z? This is very important, as reprocessing
> the message may return different results when evaluating in a context of
> a different state.
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to