Re: Detected out-of-order KTable update warnings

2020-07-10 Thread Richard Rossel
Configuring the topic to let the brokers set up the timestamps should fix my problem. Thanks for the explanation and for sharing your talk. On Fri, Jul 10, 2020 at 12:28 PM Matthias J. Sax wrote: > By default, timestamps are set by the producer. > > Thus, if different producers write to the sam

Re: Detected out-of-order KTable update warnings

2020-07-10 Thread Matthias J. Sax
By default, timestamps are set by the producer. Thus, if different producers write to the same topic-partition, their writes are interleaved and even if each individual client sends data ordered, they might end up out-of-order in the topic. As you are only interested in per-key ordering, it might

Re: Detected out-of-order KTable update warnings

2020-07-10 Thread Richard Rossel
Thanks Matthias, it makes sense, now I need to find out why the topic is not sorted by timestamp. The topic I'm using to be loaded as globalKTable is partitioned by key, and the timestamp is being handled by kafka. I have multiple clients, different machines, pushing data to that topic, maybe the

Re: Detected out-of-order KTable update warnings

2020-07-09 Thread Matthias J. Sax
If you load data into a KTable or GlobalKTable, it's expected that the data is partitioned by key, and that records with the same key have non-descending timestamps. If a record with let's say key A and timestamp 5 is put into the table, and later a record with key A and timestamp 4 is put into th

Detected out-of-order KTable update warnings

2020-07-08 Thread Richard Rossel
Hi there, I'm getting lot of this type of warning: WARN org.apache.kafka.streams.kstream.internals.KTableSource - Detected out-of-order KTable update for entity-STATE-STORE-00 at offset 65806, partition 5. It looks like the warning is generated each time a new record goes into the sour