Re: Kafka Streams - ordering grouped messages

2017-03-06 Thread Damian Guy
Hi Ofir, My advice it to handle the duplicates. As you said compaction only runs on the non-active segments. There could be duplicates in the active segment. Further, even after compaction has run there could still be duplicates. You can attempt to minimize the occurrence of duplicates by adjustin

Re: Kafka Streams - ordering grouped messages

2017-03-05 Thread Ofir Sharony
Thanks guys, I would like to continue where we stopped (late arriving records): As I understand, the best practice to handle late arriving records is enabling Kafka log compaction, thus keeping only the latest record of a certain key. As log compaction starts to do its magic only on non-active se

Re: Kafka Streams - ordering grouped messages

2017-03-01 Thread Matthias J. Sax
Just wanted to add, that there is always the potential about late arriving records, and thus, ordering by timestamp will never be perfect... You should rather try to design you application in a way such that it can handle out-of-order data gracefully and try to avoid the necessity of ordering reco

Re: Kafka Streams - ordering grouped messages

2017-03-01 Thread Damian Guy
You could implement your own based sorting algorithm using the low level processor api, i.e, you have a processor that keeps a sorted list of records and then, periodically, perhaps on punctuate, it emits the sorted messages downstream. You could do something like: builder.stream("topic").tran

Re: Kafka Streams - ordering grouped messages

2017-03-01 Thread Ofir Sharony
Is there any way to sort grouped records before sending them to the reducer? *Ofir Sharony* BackEnd Tech Lead Mobile: +972-54-7560277 | ofir.shar...@myheritage.com | www.myheritage.com MyHeritage Ltd., 3 Ariel Sharon St., Or Yehuda 60250, Israel

Re: Kafka Streams - ordering grouped messages

2017-03-01 Thread Damian Guy
Hi, The TimestampExtractor won't effect the order the records arrive in. It just provides a way for developers to use a timestamp other than the default. Thanks, Damian On Wed, 1 Mar 2017 at 12:34 Ofir Sharony wrote: > Hi, > > I have the following code on a stream: > > .selectKey(...) > .group

Kafka Streams - ordering grouped messages

2017-03-01 Thread Ofir Sharony
Hi, I have the following code on a stream: .selectKey(...) .groupByKey(...) .reduce(...) The records arrived to the Reducer function in the same order they were consumed from Kafka I have implemented a TimestampExtractor, extracting the wanted timestamp from each record, but unfortunately this d