Hi Tony,

I'll try to address your questions below:

   1. While it's not technically "wrong" to commit for each record, you
   need to keep in mind that calling commit has resource implications for
   Kafka Streams.  Here's a link to our FAQ describing what happens during a
   commit
   
https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-WhenIcommitmyprocessstate,whatdoestheStreamslibrarydoandhowitaffectsmyapplication'sperformance
   ?
   2. While you can keep track of the number of records and call commit
   manually after N records, Kafka Streams executes commit on regular
   intervals as determined by the StreamsConfig.COMMIT_INTERVAL_MS_CONFIG
   with the default value of 30 seconds.  So depending on your use case it
   might be more advantageous to figure out the optimal time interval for
   committing and set the configuration accordingly and let the framework
   handle the commit calls.
   3.  Not a specific issue with 1.1.0 as far as I know.

HTH,
Bill


On Fri, Apr 27, 2018 at 11:43 AM, Tony John <tonyjohnant...@gmail.com>
wrote:

> Hi All,
>
> I was trying to switch to the latest version of streams (1.1.0) and
> started seeing a significant drop in performance of the application. I was
> using 0.11.0.2 before. After doing some checks I found that the choking
> point was Rocksdb flush which contributes almost 80% of the CPU time (PFA
> the screenshots). One thing which I was doing earlier with the application
> was that I was doing a context.commit() from the Processor's process method
> for each record which gets processed. After doing some throttling on this
> and restricting the commit to every 100K records, I could see that the
> performance was on par with the previous version. So below are my queries
>
>
>    1. Is it wrong to do the store commits for each record which gets
>    processed?
>    2. Are there any other configurations which I need to make in order to
>    get rid of this other than throttling the commits
>    3. Or is it actually an issue with the 1.1.0, which I don't think will
>    be the case as I haven't seen anyone else reporting this so far.
>
> Please suggest.
>
>
>
>
> Thanks,
> Tony
>

Reply via email to