You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by "Matthias J. Sax" <mj...@apache.org> on 2022/01/04 22:27:08 UTC

Re: Kafka Streams - one topic moves faster the other one

If you observer timestamps based synchronization issues, you might also 
consider to switch to 3.0 release, that closes a few more gaps to this end.

Cf 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-695%3A+Further+Improve+Kafka+Streams+Timestamp+Synchronization


-Matthias

On 12/29/21 7:22 PM, Luke Chen wrote:
> Hi Miguel,
> 
> Yes, the grace period is the solution to fix the problem.
> 
> Alternatively, you can try to set a higher value for "max.task.idle.ms"
> configuration, because this is some kind of out-of-order data.
> Let's say, A topic has 1 record per second (fast), B topic has 1 record per
> minute (slow).
> You can set the "max.task.idle.ms" as 60 seconds or higher, to force the
> stream to wait for 1 minute for the empty topic B, before processing the
> records.
> 
>  From the document:
> https://kafka.apache.org/30/documentation/streams/developer-guide/config-streams#max-task-idle-ms
> 
> *Any config value greater than zero indicates the number of extra
> milliseconds that Streams will wait if it has a caught-up but empty
> partition. In other words, this is the amount of time to wait for new data
> to be produced to the input partitions to ensure in-order processing of
> data in the event of a slow producer. *
> 
> Hope it helps.
> 
> Thank you.
> Luke
> 
> On Thu, Dec 30, 2021 at 2:52 AM Miguel González <mi...@klar.mx>
> wrote:
> 
>> Hi team
>>
>> So I ran into a complicated issue, something which I believe Kafka Streams
>> is not prepared for.
>>
>> Basically my app is reading from two topics and joining them.
>>
>> But when testing the in my staging environment I found, one topic moves
>> faster than the other one basically pushing stream time forward
>>
>> Some partitions are even months apart. I found a question on SO detailing
>> something similar:
>>
>> https://stackoverflow.com/questions/69126351/bulk-processing-data-through-a-join-with-kafka-streams-results-in-skipping-reco
>>
>> The problem for me is that joins are no longer working. Setting a huge
>> grace period has somehow alleviated the problem for now but I don't that
>> that's the right approach and not all events join at the end anyway? Have
>> other users faced something similar, if so, how can it be resolved? Can we
>> somehow delay the processing to make them aligned some how
>>
>> thanks
>> - Miguel
>>
>