Re: kafka streams commit.interval.ms for at-least-once too high

2021-10-06 Thread Matthias J. Sax
If you build it manually / from-scratch using plain consumer/producer, it is your responsibility to avoid duplicates and/or data loss for a clean shutdown case or a rebalance. That is, why we recommend to use Kafka Streams for a consumer-process-produce pattern, as it does a lot of heavy

Re: kafka streams commit.interval.ms for at-least-once too high

2021-10-06 Thread Pushkar Deole
Matthias, Good to hear on this part that kafka streams handle this internally : "If a rebalance/shutdown is triggered, Kafka Streams will stop processing new records and just finish processing all in-flight records. Afterwards, a commit happens right away for all fully processed records." Since

Re: kafka streams commit.interval.ms for at-least-once too high

2021-10-05 Thread Matthias J. Sax
- By producer config, i hope you mean batching and other settings that will hold off producing of events. Correct me if i'm wrong Correct. - Not sure what you mean by throughput here, which configuration would dictate that? I referred to input topic throughput. If you have higher/lower

Re: kafka streams commit.interval.ms for at-least-once too high

2021-10-05 Thread Pushkar Deole
Matthias, On your response "For at-least-once, you would still get output continuously, depending on throughput and producer configs" - Not sure what you mean by throughput here, which configuration would dictate that? - By producer config, i hope you mean batching and other settings that will

Re: kafka streams commit.interval.ms for at-least-once too high

2021-10-05 Thread Matthias J. Sax
The main motivation for a shorter commit interval for EOS is end-to-end-latency. A Topology could consist of multiple sub-topologies and the end-to-end-latency for the EOS case is roughly commit-interval times number-of-subtopologies. For regular rebalances/restarts, a longer commit interval

kafka streams commit.interval.ms for at-least-once too high

2021-10-04 Thread Pushkar Deole
Hi All, I am looking into the commit.interval.ms in kafka streams which says that it is the time interval at which streams would commit offsets to source topics. However for exactly-once guarantee, default of this time is 100ms whereas for at-least-once it is 3ms (i.e. 30sec) Why is there