You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samza.apache.org by David Yu <da...@optimizely.com> on 2016/08/23 16:18:11 UTC

Debug Samza consumer lag issue

Dear Samza guys,

We are here for some debugging suggestions on our Samza job (0.10.0), which
lags behind on consumption after running for a couple of hours, regardless
of the number of containers allocated (currently 5).

Briefly, the job aggregates events into sessions (in Avro) during process()
and emits snapshots of the open sessions using window() every minute. This
graph
<https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%202016-08-23%2010.33.16.png?dl=0>
shows
you where processing started to lag (red is the number of events received
and green is the number of event processed). The end result is a steady
increase of the consumer lag
<https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%202016-08-23%2010.19.27.png?dl=0>.
What we are trying to track down is where the performance bottleneck is.
But it's unclear at the moment if that's in Samza or in Kafka.

What we know so far:

   - Kafka producer seems to take a while writing to the downstream topic
   (changelog and session snapshots) shown by various timers. Not sure which
   numbers are critical but here are the producer metrics
   <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%202016-08-23%2010.57.33.png?dl=0>
from
   one container.
   - avg windowing duration peaks at one point during the day (due to the
   number of open sessions) but everything is still sub-seconds
   <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%202016-08-23%2010.44.19.png?dl=0>
   .
   - our Kafka cluster doesn't seem to be overloaded
   <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%202016-08-23%2010.48.25.png?dl=0>
    with writes < 60MB/s across all three brokers

From all we know, we suspected that the bottleneck happens at producing to
Kafka. But we need some help confirming that.

Any suggestion is appreciated.

David

Re: Debug Samza consumer lag issue

Posted by David Yu <da...@optimizely.com>.
Make sense. Thanks for the help, Jake!

On Wed, Aug 24, 2016 at 5:11 PM Jacob Maes <ja...@gmail.com> wrote:

> We don't have any hard guidelines around that metric just because there are
> no hard rules that work for every job. For example, some jobs are very
> bursty and need to keep up with huge traffic ramp-ups even though they're
> underutilized the rest of the time.
>
> That said, yes, I have used that metric to determine whether a job has too
> much parallelism. But it was a job that had very stable throughput patterns
> and didn't have any major time spent in the window or commit methods, which
> could cause periodic spikes in utilization.
>
>
>
> On Wed, Aug 24, 2016 at 2:55 PM, David Yu <da...@optimizely.com> wrote:
>
> > Interesting.
> >
> > To me, "event-loop-utilization" looks like a good indicator that shows us
> > how busy the containers are. Is it safe to use this metric as a reference
> > when we need to scale out/in our job? For example, if I'm seeing around
> 0.3
> > utilization most of the time, maybe I can decrease the # of containers
> and
> > save some resources?
> >
> > Thanks,
> > David
> >
> > On Wed, Aug 24, 2016 at 1:27 PM Jacob Maes <ja...@gmail.com> wrote:
> >
> > > >
> > > > Based on what you have described, the following should be true in
> > 0.10.1:
> > > > event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
> > > > commit-ns (if necessary)
> > >
> > > Yes, plus any time (e.g. due to an unlucky GC at just the right moment)
> > > that happens outside those timers.  And no "if necessary" for window or
> > > commit. There will be a small value for those methods even if they
> don't
> > do
> > > anything significant because the timer runs even for no-ops
> > >
> > > Since you're on 10.1, there's another useful metric
> > > "event-loop-utilization", which represents
> > > (process-ns+window-ns+commit-ns)/event-loop-ns
> > > (as you defined it). In other words, the proportion of time spend
> working
> > > vs waiting.
> > >
> > > On Wed, Aug 24, 2016 at 10:18 AM, David Yu <da...@optimizely.com>
> > > wrote:
> > >
> > > > Great. It all makes sense now.
> > > >
> > > > With the SSD fix, we also upgrade to 0.10.1. So we should see pretty
> > > > consistent process-ns (which we do).
> > > >
> > > > Based on what you have described, the following should be true in
> > 0.10.1:
> > > > event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
> > > > commit-ns (if necessary)
> > > >
> > > > Is this correct?
> > > > Thanks,
> > > > David
> > > >
> > > > On Wed, Aug 24, 2016 at 11:27 AM Jacob Maes <ja...@gmail.com>
> > > wrote:
> > > >
> > > > > A couple other notes.
> > > > >
> > > > > Prior to Samza 10.1, the choose-ns was part of process-ns. So when
> > > > > choose-ns and process-ns are both high (around 10,000,000 == 10ms,
> > > which
> > > > is
> > > > > the default poll timeout), that usually means the task is caught
> up.
> > In
> > > > > Samza 10.1 the same is true if ONLY choose-ns is high. process-ns
> is
> > > > always
> > > > > the time spent in the process() method.
> > > > >
> > > > > Based on the above, the metric numbers you provided after the SSD
> fix
> > > all
> > > > > look reasonable. They're all sub-millisecond and since choose-ns
> and
> > > > > process-ns are low, it seems that the container is chewing through
> > > > messages
> > > > > at a good rate.
> > > > >
> > > > > So I would conclude that the SSD fix was probably the right one and
> > it
> > > > just
> > > > > took the job a while to catch up to the backlog of messages. Once
> it
> > > > caught
> > > > > up, the choose-ns and process-ns increased, which is normal when
> the
> > > > > processor is waiting for new messages.
> > > > >
> > > > > -Jake
> > > > >
> > > > > On Wed, Aug 24, 2016 at 9:05 AM, Jacob Maes <ja...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hey David,
> > > > > >
> > > > > > Answering the most recent question first, since it's also the
> > > easiest.
> > > > > :-)
> > > > > >
> > > > > > Is choose-ns the total number of ms used to choose a message from
> > the
> > > > > input
> > > > > >> stream? What are some gating factors (e.g. serialization?) for
> > this
> > > > > >> metric?
> > > > > >
> > > > > > It's the amount of time the event loop spent getting new
> messsages
> > > for
> > > > > > process(). It includes deserialization time and poll time which
> we
> > > > added
> > > > > > new metrics for, in Samza 10.1. Typically deserialization time is
> > > > pretty
> > > > > > consistent, so when you see a spike in choose-ns, it's usually
> > > because
> > > > > the
> > > > > > event loop is waiting for new messages. The two most common cases
> > > when
> > > > > it's
> > > > > > waiting are:
> > > > > > 1. There are no new messages in the topic partition. This is good
> > > > because
> > > > > > it means the processor is caught up.
> > > > > > 2. The consumer is slow and/or the buffer isn't large enough so
> the
> > > > > > BrokerProxy isn't able to keep enough messages buffered to keep
> the
> > > > event
> > > > > > loop busy. This is uncommon because the buffer is defaulted to
> > 50,000
> > > > > > messages, which should be plenty. But if it happens, it's bad. To
> > > > control
> > > > > > this behavior, see the following properties in the config table (
> > > > > > http://samza.apache.org/learn/documentation/0.10/jobs/
> > > > > > configuration-table.html)
> > > > > > systems.system-name.samza.fetch.threshold
> > > > > > task.poll.interval.ms
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Wed, Aug 24, 2016 at 8:52 AM, David Yu <
> david.yu@optimizely.com
> > >
> > > > > wrote:
> > > > > >
> > > > > >> More updates:
> > > > > >> 1. process-envelopes rate finally stabilized and converged.
> > Consumer
> > > > lag
> > > > > >> is
> > > > > >> down to zero.
> > > > > >> 2. avg choose-ns across containers dropped overtime
> > > > > >> <https://www.dropbox.com/s/z4iiilvd7c1wrjc/Screenshot%202016
> > > > > >> -08-24%2010.46.22.png?dl=0>,
> > > > > >> which I assume is a good thing.
> > > > > >>
> > > > > >> My question:
> > > > > >> Is choose-ns the total number of ms used to choose a message
> from
> > > the
> > > > > >> input
> > > > > >> stream? What are some gating factors (e.g. serialization?) for
> > this
> > > > > >> metric?
> > > > > >>
> > > > > >> Thanks,
> > > > > >> David
> > > > > >>
> > > > > >> On Wed, Aug 24, 2016 at 12:34 AM David Yu <
> > david.yu@optimizely.com>
> > > > > >> wrote:
> > > > > >>
> > > > > >> > Some metric updates:
> > > > > >> > 1. We started seeing some containers with a higher choose-ns
> > > > > >> > <https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%202016
> > > > > >> -08-24%2000.26.07.png?dl=0>.
> > > > > >> > Not sure what would be the cause of this.
> > > > > >> > 2. We are seeing very different process-envelopes values
> across
> > > > > >> containers
> > > > > >> > <https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%202016
> > > > > >> -08-24%2000.21.05.png?dl=0>
> > > > > >> > .
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> > On Tue, Aug 23, 2016 at 5:56 PM David Yu <
> > david.yu@optimizely.com
> > > >
> > > > > >> wrote:
> > > > > >> >
> > > > > >> >> Hi, Jake,
> > > > > >> >>
> > > > > >> >> Thanks for your suggestions. Some of my answers inline:
> > > > > >> >>
> > > > > >> >> 1.
> > > > > >> >> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <
> > > jacob.maes@gmail.com>
> > > > > >> wrote:
> > > > > >> >>
> > > > > >> >>> Hey David,
> > > > > >> >>>
> > > > > >> >>> A few initial thoughts/questions:
> > > > > >> >>>
> > > > > >> >>>
> > > > > >> >>>    1. Is this job using RocksDB to store the aggregations?
> If
> > > so,
> > > > is
> > > > > >> it
> > > > > >> >>>    running on a machine with SSDs? We've seen a few
> > performance
> > > > > issues
> > > > > >> >>> related
> > > > > >> >>>    to RocksDB.
> > > > > >> >>>       1. Not running on SSD causes slowness on disk
> > > > > >> >>
> > > > > >> >>  - [David] This definitely pointed me to the right direction
> in
> > > my
> > > > > >> >> investigation. We do use RocksDB and just realized that our
> > YARN
> > > > > >> cluster is
> > > > > >> >> using c3.xlarge EC2 instances and is using a mixture of EBS
> and
> > > > local
> > > > > >> SSD
> > > > > >> >> storage. After digging around, we noticed that some
> containers
> > > were
> > > > > >> >> persisting their KV stores in SSD while others were using
> EBS.
> > We
> > > > > just
> > > > > >> >> updated our YARN config to use SSD only before redeployed our
> > > jobs.
> > > > > So
> > > > > >> far
> > > > > >> >> everything looks good. Will report back on the performance
> > > update.
> > > > > >> >>
> > > > > >> >>>       2. Prior to Samza 10.1, samza would excessively flush
> > the
> > > > > store
> > > > > >> to
> > > > > >> >>>       disk, causing RocksDB compaction issues (stalls) -
> > > SAMZA-957
> > > > > >> >>>
> > > > > >> >> - [David] We did notice that the log cleaner thread died on
> one
> > > of
> > > > > our
> > > > > >> >> brokers. Not sure if this was the same problem you pointed
> out.
> > > > > >> Following
> > > > > >> >> errors are logged:
> > > > > >> >>
> > > > > >> >> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
> > > > > >> >> [kafka-log-cleaner-thread-0], Error due to
> > > > > >> >>
> > > > > >> >> java.lang.IllegalArgumentException: requirement failed:
> > 5865800
> > > > > >> messages
> > > > > >> >> in segment
> > > > > session-store-2.0-tickets-changelog-9/00000000000009548937.
> > > > > >> log
> > > > > >> >> but offset map can fit only 5033164. You can increase
> > > > > >> >> log.cleaner.dedupe.buffer.size or decrease
> log.cleaner.threads
> > > > > >> >>
> > > > > >> >>         at scala.Predef$.require(Predef.scala:219)
> > > > > >> >>
> > > > > >> >> We had to cleanup the changelog topic and restart the broker
> to
> > > > bring
> > > > > >> >> back the cleaner thread.
> > > > > >> >>
> > > > > >> >>>       3. When the RocksDB store is used as a queue, the
> > iterator
> > > > can
> > > > > >> >>> suffer
> > > > > >> >>>       performance issues due to RocksDBs tombstoning. (
> > > > > >> >>>
> > > > > >> >>>
> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Ser
> > > > > >> vice-Using-RocksDB
> > > > > >> >>>       )
> > > > > >> >>>
> > > > > >> >> - [David] We use RocksDB to keep track of opening sessions
> and
> > > use
> > > > > >> >> sessionId (a random hash) as the key. In that sense, this
> does
> > > not
> > > > > >> sound
> > > > > >> >> like a queue. But we do iterate and delete closed sessions
> > during
> > > > > >> windowing
> > > > > >> >> on a minute by minute basis.
> > > > > >> >>
> > > > > >> >>    2. Is the "messages-behind-high-watermark" metric
> non-zero?
> > > > > >> >>>
> > > > > >> >> -[David] Yes.
> > > > > >> >>
> > > > > >> >>>    3. The SamzaContainerMetrics might be useful too.
> > > Particularly
> > > > > >> >>>    "choose-ns" and "commit-ns"
> > > > > >> >>>
> > > > > >> >> -[David] We are seeing the following from one of the
> containers
> > > > > (after
> > > > > >> >> the SSD fix mentioned above):
> > > > > >> >> choose-ns=61353
> > > > > >> >> commit-ns=306328 (what does this metric indicate? Is this in
> > ms?)
> > > > > >> >> process-ns=248260
> > > > > >> >> window-ns=150717
> > > > > >> >>
> > > > > >> >>>    4. The only time I've personally seen slowness on the
> > > producer
> > > > is
> > > > > >> if
> > > > > >> >>>    it's configured for acks="all". What is the producer
> config
> > > > from
> > > > > >> the
> > > > > >> >>> log?
> > > > > >> >>>
> > > > > >> >> - [David] We did not override this. So should be the default
> > > value
> > > > > >> (1?).
> > > > > >> >>
> > > > > >> >>    5. The window time is high, but since it's only called
> once
> > > per
> > > > > >> minute,
> > > > > >> >>>    it looks like it only represents 1% of the event loop
> > > > > utilization.
> > > > > >> So
> > > > > >> >>> I
> > > > > >> >>>    don't think that's a smoking gun.
> > > > > >> >>>
> > > > > >> >>> -Jake
> > > > > >> >>>
> > > > > >> >>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <
> > > > david.yu@optimizely.com>
> > > > > >> >>> wrote:
> > > > > >> >>>
> > > > > >> >>> > Dear Samza guys,
> > > > > >> >>> >
> > > > > >> >>> > We are here for some debugging suggestions on our Samza
> job
> > > > > >> (0.10.0),
> > > > > >> >>> which
> > > > > >> >>> > lags behind on consumption after running for a couple of
> > > hours,
> > > > > >> >>> regardless
> > > > > >> >>> > of the number of containers allocated (currently 5).
> > > > > >> >>> >
> > > > > >> >>> > Briefly, the job aggregates events into sessions (in Avro)
> > > > during
> > > > > >> >>> process()
> > > > > >> >>> > and emits snapshots of the open sessions using window()
> > every
> > > > > >> minute.
> > > > > >> >>> This
> > > > > >> >>> > graph
> > > > > >> >>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
> > > > > >> >>> > 202016-08-23%2010.33.16.png?dl=0>
> > > > > >> >>> > shows
> > > > > >> >>> > you where processing started to lag (red is the number of
> > > events
> > > > > >> >>> received
> > > > > >> >>> > and green is the number of event processed). The end
> result
> > > is a
> > > > > >> steady
> > > > > >> >>> > increase of the consumer lag
> > > > > >> >>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
> > > > > >> >>> > 202016-08-23%2010.19.27.png?dl=0>.
> > > > > >> >>> > What we are trying to track down is where the performance
> > > > > bottleneck
> > > > > >> >>> is.
> > > > > >> >>> > But it's unclear at the moment if that's in Samza or in
> > Kafka.
> > > > > >> >>> >
> > > > > >> >>> > What we know so far:
> > > > > >> >>> >
> > > > > >> >>> >    - Kafka producer seems to take a while writing to the
> > > > > downstream
> > > > > >> >>> topic
> > > > > >> >>> >    (changelog and session snapshots) shown by various
> > timers.
> > > > Not
> > > > > >> sure
> > > > > >> >>> > which
> > > > > >> >>> >    numbers are critical but here are the producer metrics
> > > > > >> >>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
> > > > > >> >>> > 202016-08-23%2010.57.33.png?dl=0>
> > > > > >> >>> > from
> > > > > >> >>> >    one container.
> > > > > >> >>> >    - avg windowing duration peaks at one point during the
> > day
> > > > (due
> > > > > >> to
> > > > > >> >>> the
> > > > > >> >>> >    number of open sessions) but everything is still
> > > sub-seconds
> > > > > >> >>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
> > > > > >> >>> > 202016-08-23%2010.44.19.png?dl=0>
> > > > > >> >>> >    .
> > > > > >> >>> >    - our Kafka cluster doesn't seem to be overloaded
> > > > > >> >>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
> > > > > >> >>> > 202016-08-23%2010.48.25.png?dl=0>
> > > > > >> >>> >     with writes < 60MB/s across all three brokers
> > > > > >> >>> >
> > > > > >> >>> > From all we know, we suspected that the bottleneck happens
> > at
> > > > > >> >>> producing to
> > > > > >> >>> > Kafka. But we need some help confirming that.
> > > > > >> >>> >
> > > > > >> >>> > Any suggestion is appreciated.
> > > > > >> >>> >
> > > > > >> >>> > David
> > > > > >> >>> >
> > > > > >> >>>
> > > > > >> >>
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Debug Samza consumer lag issue

Posted by Jacob Maes <ja...@gmail.com>.
We don't have any hard guidelines around that metric just because there are
no hard rules that work for every job. For example, some jobs are very
bursty and need to keep up with huge traffic ramp-ups even though they're
underutilized the rest of the time.

That said, yes, I have used that metric to determine whether a job has too
much parallelism. But it was a job that had very stable throughput patterns
and didn't have any major time spent in the window or commit methods, which
could cause periodic spikes in utilization.



On Wed, Aug 24, 2016 at 2:55 PM, David Yu <da...@optimizely.com> wrote:

> Interesting.
>
> To me, "event-loop-utilization" looks like a good indicator that shows us
> how busy the containers are. Is it safe to use this metric as a reference
> when we need to scale out/in our job? For example, if I'm seeing around 0.3
> utilization most of the time, maybe I can decrease the # of containers and
> save some resources?
>
> Thanks,
> David
>
> On Wed, Aug 24, 2016 at 1:27 PM Jacob Maes <ja...@gmail.com> wrote:
>
> > >
> > > Based on what you have described, the following should be true in
> 0.10.1:
> > > event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
> > > commit-ns (if necessary)
> >
> > Yes, plus any time (e.g. due to an unlucky GC at just the right moment)
> > that happens outside those timers.  And no "if necessary" for window or
> > commit. There will be a small value for those methods even if they don't
> do
> > anything significant because the timer runs even for no-ops
> >
> > Since you're on 10.1, there's another useful metric
> > "event-loop-utilization", which represents
> > (process-ns+window-ns+commit-ns)/event-loop-ns
> > (as you defined it). In other words, the proportion of time spend working
> > vs waiting.
> >
> > On Wed, Aug 24, 2016 at 10:18 AM, David Yu <da...@optimizely.com>
> > wrote:
> >
> > > Great. It all makes sense now.
> > >
> > > With the SSD fix, we also upgrade to 0.10.1. So we should see pretty
> > > consistent process-ns (which we do).
> > >
> > > Based on what you have described, the following should be true in
> 0.10.1:
> > > event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
> > > commit-ns (if necessary)
> > >
> > > Is this correct?
> > > Thanks,
> > > David
> > >
> > > On Wed, Aug 24, 2016 at 11:27 AM Jacob Maes <ja...@gmail.com>
> > wrote:
> > >
> > > > A couple other notes.
> > > >
> > > > Prior to Samza 10.1, the choose-ns was part of process-ns. So when
> > > > choose-ns and process-ns are both high (around 10,000,000 == 10ms,
> > which
> > > is
> > > > the default poll timeout), that usually means the task is caught up.
> In
> > > > Samza 10.1 the same is true if ONLY choose-ns is high. process-ns is
> > > always
> > > > the time spent in the process() method.
> > > >
> > > > Based on the above, the metric numbers you provided after the SSD fix
> > all
> > > > look reasonable. They're all sub-millisecond and since choose-ns and
> > > > process-ns are low, it seems that the container is chewing through
> > > messages
> > > > at a good rate.
> > > >
> > > > So I would conclude that the SSD fix was probably the right one and
> it
> > > just
> > > > took the job a while to catch up to the backlog of messages. Once it
> > > caught
> > > > up, the choose-ns and process-ns increased, which is normal when the
> > > > processor is waiting for new messages.
> > > >
> > > > -Jake
> > > >
> > > > On Wed, Aug 24, 2016 at 9:05 AM, Jacob Maes <ja...@gmail.com>
> > > wrote:
> > > >
> > > > > Hey David,
> > > > >
> > > > > Answering the most recent question first, since it's also the
> > easiest.
> > > > :-)
> > > > >
> > > > > Is choose-ns the total number of ms used to choose a message from
> the
> > > > input
> > > > >> stream? What are some gating factors (e.g. serialization?) for
> this
> > > > >> metric?
> > > > >
> > > > > It's the amount of time the event loop spent getting new messsages
> > for
> > > > > process(). It includes deserialization time and poll time which we
> > > added
> > > > > new metrics for, in Samza 10.1. Typically deserialization time is
> > > pretty
> > > > > consistent, so when you see a spike in choose-ns, it's usually
> > because
> > > > the
> > > > > event loop is waiting for new messages. The two most common cases
> > when
> > > > it's
> > > > > waiting are:
> > > > > 1. There are no new messages in the topic partition. This is good
> > > because
> > > > > it means the processor is caught up.
> > > > > 2. The consumer is slow and/or the buffer isn't large enough so the
> > > > > BrokerProxy isn't able to keep enough messages buffered to keep the
> > > event
> > > > > loop busy. This is uncommon because the buffer is defaulted to
> 50,000
> > > > > messages, which should be plenty. But if it happens, it's bad. To
> > > control
> > > > > this behavior, see the following properties in the config table (
> > > > > http://samza.apache.org/learn/documentation/0.10/jobs/
> > > > > configuration-table.html)
> > > > > systems.system-name.samza.fetch.threshold
> > > > > task.poll.interval.ms
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Aug 24, 2016 at 8:52 AM, David Yu <david.yu@optimizely.com
> >
> > > > wrote:
> > > > >
> > > > >> More updates:
> > > > >> 1. process-envelopes rate finally stabilized and converged.
> Consumer
> > > lag
> > > > >> is
> > > > >> down to zero.
> > > > >> 2. avg choose-ns across containers dropped overtime
> > > > >> <https://www.dropbox.com/s/z4iiilvd7c1wrjc/Screenshot%202016
> > > > >> -08-24%2010.46.22.png?dl=0>,
> > > > >> which I assume is a good thing.
> > > > >>
> > > > >> My question:
> > > > >> Is choose-ns the total number of ms used to choose a message from
> > the
> > > > >> input
> > > > >> stream? What are some gating factors (e.g. serialization?) for
> this
> > > > >> metric?
> > > > >>
> > > > >> Thanks,
> > > > >> David
> > > > >>
> > > > >> On Wed, Aug 24, 2016 at 12:34 AM David Yu <
> david.yu@optimizely.com>
> > > > >> wrote:
> > > > >>
> > > > >> > Some metric updates:
> > > > >> > 1. We started seeing some containers with a higher choose-ns
> > > > >> > <https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%202016
> > > > >> -08-24%2000.26.07.png?dl=0>.
> > > > >> > Not sure what would be the cause of this.
> > > > >> > 2. We are seeing very different process-envelopes values across
> > > > >> containers
> > > > >> > <https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%202016
> > > > >> -08-24%2000.21.05.png?dl=0>
> > > > >> > .
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Tue, Aug 23, 2016 at 5:56 PM David Yu <
> david.yu@optimizely.com
> > >
> > > > >> wrote:
> > > > >> >
> > > > >> >> Hi, Jake,
> > > > >> >>
> > > > >> >> Thanks for your suggestions. Some of my answers inline:
> > > > >> >>
> > > > >> >> 1.
> > > > >> >> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <
> > jacob.maes@gmail.com>
> > > > >> wrote:
> > > > >> >>
> > > > >> >>> Hey David,
> > > > >> >>>
> > > > >> >>> A few initial thoughts/questions:
> > > > >> >>>
> > > > >> >>>
> > > > >> >>>    1. Is this job using RocksDB to store the aggregations? If
> > so,
> > > is
> > > > >> it
> > > > >> >>>    running on a machine with SSDs? We've seen a few
> performance
> > > > issues
> > > > >> >>> related
> > > > >> >>>    to RocksDB.
> > > > >> >>>       1. Not running on SSD causes slowness on disk
> > > > >> >>
> > > > >> >>  - [David] This definitely pointed me to the right direction in
> > my
> > > > >> >> investigation. We do use RocksDB and just realized that our
> YARN
> > > > >> cluster is
> > > > >> >> using c3.xlarge EC2 instances and is using a mixture of EBS and
> > > local
> > > > >> SSD
> > > > >> >> storage. After digging around, we noticed that some containers
> > were
> > > > >> >> persisting their KV stores in SSD while others were using EBS.
> We
> > > > just
> > > > >> >> updated our YARN config to use SSD only before redeployed our
> > jobs.
> > > > So
> > > > >> far
> > > > >> >> everything looks good. Will report back on the performance
> > update.
> > > > >> >>
> > > > >> >>>       2. Prior to Samza 10.1, samza would excessively flush
> the
> > > > store
> > > > >> to
> > > > >> >>>       disk, causing RocksDB compaction issues (stalls) -
> > SAMZA-957
> > > > >> >>>
> > > > >> >> - [David] We did notice that the log cleaner thread died on one
> > of
> > > > our
> > > > >> >> brokers. Not sure if this was the same problem you pointed out.
> > > > >> Following
> > > > >> >> errors are logged:
> > > > >> >>
> > > > >> >> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
> > > > >> >> [kafka-log-cleaner-thread-0], Error due to
> > > > >> >>
> > > > >> >> java.lang.IllegalArgumentException: requirement failed:
> 5865800
> > > > >> messages
> > > > >> >> in segment
> > > > session-store-2.0-tickets-changelog-9/00000000000009548937.
> > > > >> log
> > > > >> >> but offset map can fit only 5033164. You can increase
> > > > >> >> log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
> > > > >> >>
> > > > >> >>         at scala.Predef$.require(Predef.scala:219)
> > > > >> >>
> > > > >> >> We had to cleanup the changelog topic and restart the broker to
> > > bring
> > > > >> >> back the cleaner thread.
> > > > >> >>
> > > > >> >>>       3. When the RocksDB store is used as a queue, the
> iterator
> > > can
> > > > >> >>> suffer
> > > > >> >>>       performance issues due to RocksDBs tombstoning. (
> > > > >> >>>
> > > > >> >>> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Ser
> > > > >> vice-Using-RocksDB
> > > > >> >>>       )
> > > > >> >>>
> > > > >> >> - [David] We use RocksDB to keep track of opening sessions and
> > use
> > > > >> >> sessionId (a random hash) as the key. In that sense, this does
> > not
> > > > >> sound
> > > > >> >> like a queue. But we do iterate and delete closed sessions
> during
> > > > >> windowing
> > > > >> >> on a minute by minute basis.
> > > > >> >>
> > > > >> >>    2. Is the "messages-behind-high-watermark" metric non-zero?
> > > > >> >>>
> > > > >> >> -[David] Yes.
> > > > >> >>
> > > > >> >>>    3. The SamzaContainerMetrics might be useful too.
> > Particularly
> > > > >> >>>    "choose-ns" and "commit-ns"
> > > > >> >>>
> > > > >> >> -[David] We are seeing the following from one of the containers
> > > > (after
> > > > >> >> the SSD fix mentioned above):
> > > > >> >> choose-ns=61353
> > > > >> >> commit-ns=306328 (what does this metric indicate? Is this in
> ms?)
> > > > >> >> process-ns=248260
> > > > >> >> window-ns=150717
> > > > >> >>
> > > > >> >>>    4. The only time I've personally seen slowness on the
> > producer
> > > is
> > > > >> if
> > > > >> >>>    it's configured for acks="all". What is the producer config
> > > from
> > > > >> the
> > > > >> >>> log?
> > > > >> >>>
> > > > >> >> - [David] We did not override this. So should be the default
> > value
> > > > >> (1?).
> > > > >> >>
> > > > >> >>    5. The window time is high, but since it's only called once
> > per
> > > > >> minute,
> > > > >> >>>    it looks like it only represents 1% of the event loop
> > > > utilization.
> > > > >> So
> > > > >> >>> I
> > > > >> >>>    don't think that's a smoking gun.
> > > > >> >>>
> > > > >> >>> -Jake
> > > > >> >>>
> > > > >> >>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <
> > > david.yu@optimizely.com>
> > > > >> >>> wrote:
> > > > >> >>>
> > > > >> >>> > Dear Samza guys,
> > > > >> >>> >
> > > > >> >>> > We are here for some debugging suggestions on our Samza job
> > > > >> (0.10.0),
> > > > >> >>> which
> > > > >> >>> > lags behind on consumption after running for a couple of
> > hours,
> > > > >> >>> regardless
> > > > >> >>> > of the number of containers allocated (currently 5).
> > > > >> >>> >
> > > > >> >>> > Briefly, the job aggregates events into sessions (in Avro)
> > > during
> > > > >> >>> process()
> > > > >> >>> > and emits snapshots of the open sessions using window()
> every
> > > > >> minute.
> > > > >> >>> This
> > > > >> >>> > graph
> > > > >> >>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
> > > > >> >>> > 202016-08-23%2010.33.16.png?dl=0>
> > > > >> >>> > shows
> > > > >> >>> > you where processing started to lag (red is the number of
> > events
> > > > >> >>> received
> > > > >> >>> > and green is the number of event processed). The end result
> > is a
> > > > >> steady
> > > > >> >>> > increase of the consumer lag
> > > > >> >>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
> > > > >> >>> > 202016-08-23%2010.19.27.png?dl=0>.
> > > > >> >>> > What we are trying to track down is where the performance
> > > > bottleneck
> > > > >> >>> is.
> > > > >> >>> > But it's unclear at the moment if that's in Samza or in
> Kafka.
> > > > >> >>> >
> > > > >> >>> > What we know so far:
> > > > >> >>> >
> > > > >> >>> >    - Kafka producer seems to take a while writing to the
> > > > downstream
> > > > >> >>> topic
> > > > >> >>> >    (changelog and session snapshots) shown by various
> timers.
> > > Not
> > > > >> sure
> > > > >> >>> > which
> > > > >> >>> >    numbers are critical but here are the producer metrics
> > > > >> >>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
> > > > >> >>> > 202016-08-23%2010.57.33.png?dl=0>
> > > > >> >>> > from
> > > > >> >>> >    one container.
> > > > >> >>> >    - avg windowing duration peaks at one point during the
> day
> > > (due
> > > > >> to
> > > > >> >>> the
> > > > >> >>> >    number of open sessions) but everything is still
> > sub-seconds
> > > > >> >>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
> > > > >> >>> > 202016-08-23%2010.44.19.png?dl=0>
> > > > >> >>> >    .
> > > > >> >>> >    - our Kafka cluster doesn't seem to be overloaded
> > > > >> >>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
> > > > >> >>> > 202016-08-23%2010.48.25.png?dl=0>
> > > > >> >>> >     with writes < 60MB/s across all three brokers
> > > > >> >>> >
> > > > >> >>> > From all we know, we suspected that the bottleneck happens
> at
> > > > >> >>> producing to
> > > > >> >>> > Kafka. But we need some help confirming that.
> > > > >> >>> >
> > > > >> >>> > Any suggestion is appreciated.
> > > > >> >>> >
> > > > >> >>> > David
> > > > >> >>> >
> > > > >> >>>
> > > > >> >>
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: Debug Samza consumer lag issue

Posted by David Yu <da...@optimizely.com>.
Interesting.

To me, "event-loop-utilization" looks like a good indicator that shows us
how busy the containers are. Is it safe to use this metric as a reference
when we need to scale out/in our job? For example, if I'm seeing around 0.3
utilization most of the time, maybe I can decrease the # of containers and
save some resources?

Thanks,
David

On Wed, Aug 24, 2016 at 1:27 PM Jacob Maes <ja...@gmail.com> wrote:

> >
> > Based on what you have described, the following should be true in 0.10.1:
> > event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
> > commit-ns (if necessary)
>
> Yes, plus any time (e.g. due to an unlucky GC at just the right moment)
> that happens outside those timers.  And no "if necessary" for window or
> commit. There will be a small value for those methods even if they don't do
> anything significant because the timer runs even for no-ops
>
> Since you're on 10.1, there's another useful metric
> "event-loop-utilization", which represents
> (process-ns+window-ns+commit-ns)/event-loop-ns
> (as you defined it). In other words, the proportion of time spend working
> vs waiting.
>
> On Wed, Aug 24, 2016 at 10:18 AM, David Yu <da...@optimizely.com>
> wrote:
>
> > Great. It all makes sense now.
> >
> > With the SSD fix, we also upgrade to 0.10.1. So we should see pretty
> > consistent process-ns (which we do).
> >
> > Based on what you have described, the following should be true in 0.10.1:
> > event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
> > commit-ns (if necessary)
> >
> > Is this correct?
> > Thanks,
> > David
> >
> > On Wed, Aug 24, 2016 at 11:27 AM Jacob Maes <ja...@gmail.com>
> wrote:
> >
> > > A couple other notes.
> > >
> > > Prior to Samza 10.1, the choose-ns was part of process-ns. So when
> > > choose-ns and process-ns are both high (around 10,000,000 == 10ms,
> which
> > is
> > > the default poll timeout), that usually means the task is caught up. In
> > > Samza 10.1 the same is true if ONLY choose-ns is high. process-ns is
> > always
> > > the time spent in the process() method.
> > >
> > > Based on the above, the metric numbers you provided after the SSD fix
> all
> > > look reasonable. They're all sub-millisecond and since choose-ns and
> > > process-ns are low, it seems that the container is chewing through
> > messages
> > > at a good rate.
> > >
> > > So I would conclude that the SSD fix was probably the right one and it
> > just
> > > took the job a while to catch up to the backlog of messages. Once it
> > caught
> > > up, the choose-ns and process-ns increased, which is normal when the
> > > processor is waiting for new messages.
> > >
> > > -Jake
> > >
> > > On Wed, Aug 24, 2016 at 9:05 AM, Jacob Maes <ja...@gmail.com>
> > wrote:
> > >
> > > > Hey David,
> > > >
> > > > Answering the most recent question first, since it's also the
> easiest.
> > > :-)
> > > >
> > > > Is choose-ns the total number of ms used to choose a message from the
> > > input
> > > >> stream? What are some gating factors (e.g. serialization?) for this
> > > >> metric?
> > > >
> > > > It's the amount of time the event loop spent getting new messsages
> for
> > > > process(). It includes deserialization time and poll time which we
> > added
> > > > new metrics for, in Samza 10.1. Typically deserialization time is
> > pretty
> > > > consistent, so when you see a spike in choose-ns, it's usually
> because
> > > the
> > > > event loop is waiting for new messages. The two most common cases
> when
> > > it's
> > > > waiting are:
> > > > 1. There are no new messages in the topic partition. This is good
> > because
> > > > it means the processor is caught up.
> > > > 2. The consumer is slow and/or the buffer isn't large enough so the
> > > > BrokerProxy isn't able to keep enough messages buffered to keep the
> > event
> > > > loop busy. This is uncommon because the buffer is defaulted to 50,000
> > > > messages, which should be plenty. But if it happens, it's bad. To
> > control
> > > > this behavior, see the following properties in the config table (
> > > > http://samza.apache.org/learn/documentation/0.10/jobs/
> > > > configuration-table.html)
> > > > systems.system-name.samza.fetch.threshold
> > > > task.poll.interval.ms
> > > >
> > > >
> > > >
> > > > On Wed, Aug 24, 2016 at 8:52 AM, David Yu <da...@optimizely.com>
> > > wrote:
> > > >
> > > >> More updates:
> > > >> 1. process-envelopes rate finally stabilized and converged. Consumer
> > lag
> > > >> is
> > > >> down to zero.
> > > >> 2. avg choose-ns across containers dropped overtime
> > > >> <https://www.dropbox.com/s/z4iiilvd7c1wrjc/Screenshot%202016
> > > >> -08-24%2010.46.22.png?dl=0>,
> > > >> which I assume is a good thing.
> > > >>
> > > >> My question:
> > > >> Is choose-ns the total number of ms used to choose a message from
> the
> > > >> input
> > > >> stream? What are some gating factors (e.g. serialization?) for this
> > > >> metric?
> > > >>
> > > >> Thanks,
> > > >> David
> > > >>
> > > >> On Wed, Aug 24, 2016 at 12:34 AM David Yu <da...@optimizely.com>
> > > >> wrote:
> > > >>
> > > >> > Some metric updates:
> > > >> > 1. We started seeing some containers with a higher choose-ns
> > > >> > <https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%202016
> > > >> -08-24%2000.26.07.png?dl=0>.
> > > >> > Not sure what would be the cause of this.
> > > >> > 2. We are seeing very different process-envelopes values across
> > > >> containers
> > > >> > <https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%202016
> > > >> -08-24%2000.21.05.png?dl=0>
> > > >> > .
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Tue, Aug 23, 2016 at 5:56 PM David Yu <david.yu@optimizely.com
> >
> > > >> wrote:
> > > >> >
> > > >> >> Hi, Jake,
> > > >> >>
> > > >> >> Thanks for your suggestions. Some of my answers inline:
> > > >> >>
> > > >> >> 1.
> > > >> >> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <
> jacob.maes@gmail.com>
> > > >> wrote:
> > > >> >>
> > > >> >>> Hey David,
> > > >> >>>
> > > >> >>> A few initial thoughts/questions:
> > > >> >>>
> > > >> >>>
> > > >> >>>    1. Is this job using RocksDB to store the aggregations? If
> so,
> > is
> > > >> it
> > > >> >>>    running on a machine with SSDs? We've seen a few performance
> > > issues
> > > >> >>> related
> > > >> >>>    to RocksDB.
> > > >> >>>       1. Not running on SSD causes slowness on disk
> > > >> >>
> > > >> >>  - [David] This definitely pointed me to the right direction in
> my
> > > >> >> investigation. We do use RocksDB and just realized that our YARN
> > > >> cluster is
> > > >> >> using c3.xlarge EC2 instances and is using a mixture of EBS and
> > local
> > > >> SSD
> > > >> >> storage. After digging around, we noticed that some containers
> were
> > > >> >> persisting their KV stores in SSD while others were using EBS. We
> > > just
> > > >> >> updated our YARN config to use SSD only before redeployed our
> jobs.
> > > So
> > > >> far
> > > >> >> everything looks good. Will report back on the performance
> update.
> > > >> >>
> > > >> >>>       2. Prior to Samza 10.1, samza would excessively flush the
> > > store
> > > >> to
> > > >> >>>       disk, causing RocksDB compaction issues (stalls) -
> SAMZA-957
> > > >> >>>
> > > >> >> - [David] We did notice that the log cleaner thread died on one
> of
> > > our
> > > >> >> brokers. Not sure if this was the same problem you pointed out.
> > > >> Following
> > > >> >> errors are logged:
> > > >> >>
> > > >> >> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
> > > >> >> [kafka-log-cleaner-thread-0], Error due to
> > > >> >>
> > > >> >> java.lang.IllegalArgumentException: requirement failed: 5865800
> > > >> messages
> > > >> >> in segment
> > > session-store-2.0-tickets-changelog-9/00000000000009548937.
> > > >> log
> > > >> >> but offset map can fit only 5033164. You can increase
> > > >> >> log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
> > > >> >>
> > > >> >>         at scala.Predef$.require(Predef.scala:219)
> > > >> >>
> > > >> >> We had to cleanup the changelog topic and restart the broker to
> > bring
> > > >> >> back the cleaner thread.
> > > >> >>
> > > >> >>>       3. When the RocksDB store is used as a queue, the iterator
> > can
> > > >> >>> suffer
> > > >> >>>       performance issues due to RocksDBs tombstoning. (
> > > >> >>>
> > > >> >>> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Ser
> > > >> vice-Using-RocksDB
> > > >> >>>       )
> > > >> >>>
> > > >> >> - [David] We use RocksDB to keep track of opening sessions and
> use
> > > >> >> sessionId (a random hash) as the key. In that sense, this does
> not
> > > >> sound
> > > >> >> like a queue. But we do iterate and delete closed sessions during
> > > >> windowing
> > > >> >> on a minute by minute basis.
> > > >> >>
> > > >> >>    2. Is the "messages-behind-high-watermark" metric non-zero?
> > > >> >>>
> > > >> >> -[David] Yes.
> > > >> >>
> > > >> >>>    3. The SamzaContainerMetrics might be useful too.
> Particularly
> > > >> >>>    "choose-ns" and "commit-ns"
> > > >> >>>
> > > >> >> -[David] We are seeing the following from one of the containers
> > > (after
> > > >> >> the SSD fix mentioned above):
> > > >> >> choose-ns=61353
> > > >> >> commit-ns=306328 (what does this metric indicate? Is this in ms?)
> > > >> >> process-ns=248260
> > > >> >> window-ns=150717
> > > >> >>
> > > >> >>>    4. The only time I've personally seen slowness on the
> producer
> > is
> > > >> if
> > > >> >>>    it's configured for acks="all". What is the producer config
> > from
> > > >> the
> > > >> >>> log?
> > > >> >>>
> > > >> >> - [David] We did not override this. So should be the default
> value
> > > >> (1?).
> > > >> >>
> > > >> >>    5. The window time is high, but since it's only called once
> per
> > > >> minute,
> > > >> >>>    it looks like it only represents 1% of the event loop
> > > utilization.
> > > >> So
> > > >> >>> I
> > > >> >>>    don't think that's a smoking gun.
> > > >> >>>
> > > >> >>> -Jake
> > > >> >>>
> > > >> >>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <
> > david.yu@optimizely.com>
> > > >> >>> wrote:
> > > >> >>>
> > > >> >>> > Dear Samza guys,
> > > >> >>> >
> > > >> >>> > We are here for some debugging suggestions on our Samza job
> > > >> (0.10.0),
> > > >> >>> which
> > > >> >>> > lags behind on consumption after running for a couple of
> hours,
> > > >> >>> regardless
> > > >> >>> > of the number of containers allocated (currently 5).
> > > >> >>> >
> > > >> >>> > Briefly, the job aggregates events into sessions (in Avro)
> > during
> > > >> >>> process()
> > > >> >>> > and emits snapshots of the open sessions using window() every
> > > >> minute.
> > > >> >>> This
> > > >> >>> > graph
> > > >> >>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
> > > >> >>> > 202016-08-23%2010.33.16.png?dl=0>
> > > >> >>> > shows
> > > >> >>> > you where processing started to lag (red is the number of
> events
> > > >> >>> received
> > > >> >>> > and green is the number of event processed). The end result
> is a
> > > >> steady
> > > >> >>> > increase of the consumer lag
> > > >> >>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
> > > >> >>> > 202016-08-23%2010.19.27.png?dl=0>.
> > > >> >>> > What we are trying to track down is where the performance
> > > bottleneck
> > > >> >>> is.
> > > >> >>> > But it's unclear at the moment if that's in Samza or in Kafka.
> > > >> >>> >
> > > >> >>> > What we know so far:
> > > >> >>> >
> > > >> >>> >    - Kafka producer seems to take a while writing to the
> > > downstream
> > > >> >>> topic
> > > >> >>> >    (changelog and session snapshots) shown by various timers.
> > Not
> > > >> sure
> > > >> >>> > which
> > > >> >>> >    numbers are critical but here are the producer metrics
> > > >> >>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
> > > >> >>> > 202016-08-23%2010.57.33.png?dl=0>
> > > >> >>> > from
> > > >> >>> >    one container.
> > > >> >>> >    - avg windowing duration peaks at one point during the day
> > (due
> > > >> to
> > > >> >>> the
> > > >> >>> >    number of open sessions) but everything is still
> sub-seconds
> > > >> >>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
> > > >> >>> > 202016-08-23%2010.44.19.png?dl=0>
> > > >> >>> >    .
> > > >> >>> >    - our Kafka cluster doesn't seem to be overloaded
> > > >> >>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
> > > >> >>> > 202016-08-23%2010.48.25.png?dl=0>
> > > >> >>> >     with writes < 60MB/s across all three brokers
> > > >> >>> >
> > > >> >>> > From all we know, we suspected that the bottleneck happens at
> > > >> >>> producing to
> > > >> >>> > Kafka. But we need some help confirming that.
> > > >> >>> >
> > > >> >>> > Any suggestion is appreciated.
> > > >> >>> >
> > > >> >>> > David
> > > >> >>> >
> > > >> >>>
> > > >> >>
> > > >>
> > > >
> > > >
> > >
> >
>

Re: Debug Samza consumer lag issue

Posted by Jacob Maes <ja...@gmail.com>.
>
> Based on what you have described, the following should be true in 0.10.1:
> event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
> commit-ns (if necessary)

Yes, plus any time (e.g. due to an unlucky GC at just the right moment)
that happens outside those timers.  And no "if necessary" for window or
commit. There will be a small value for those methods even if they don't do
anything significant because the timer runs even for no-ops

Since you're on 10.1, there's another useful metric
"event-loop-utilization", which represents
(process-ns+window-ns+commit-ns)/event-loop-ns
(as you defined it). In other words, the proportion of time spend working
vs waiting.

On Wed, Aug 24, 2016 at 10:18 AM, David Yu <da...@optimizely.com> wrote:

> Great. It all makes sense now.
>
> With the SSD fix, we also upgrade to 0.10.1. So we should see pretty
> consistent process-ns (which we do).
>
> Based on what you have described, the following should be true in 0.10.1:
> event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
> commit-ns (if necessary)
>
> Is this correct?
> Thanks,
> David
>
> On Wed, Aug 24, 2016 at 11:27 AM Jacob Maes <ja...@gmail.com> wrote:
>
> > A couple other notes.
> >
> > Prior to Samza 10.1, the choose-ns was part of process-ns. So when
> > choose-ns and process-ns are both high (around 10,000,000 == 10ms, which
> is
> > the default poll timeout), that usually means the task is caught up. In
> > Samza 10.1 the same is true if ONLY choose-ns is high. process-ns is
> always
> > the time spent in the process() method.
> >
> > Based on the above, the metric numbers you provided after the SSD fix all
> > look reasonable. They're all sub-millisecond and since choose-ns and
> > process-ns are low, it seems that the container is chewing through
> messages
> > at a good rate.
> >
> > So I would conclude that the SSD fix was probably the right one and it
> just
> > took the job a while to catch up to the backlog of messages. Once it
> caught
> > up, the choose-ns and process-ns increased, which is normal when the
> > processor is waiting for new messages.
> >
> > -Jake
> >
> > On Wed, Aug 24, 2016 at 9:05 AM, Jacob Maes <ja...@gmail.com>
> wrote:
> >
> > > Hey David,
> > >
> > > Answering the most recent question first, since it's also the easiest.
> > :-)
> > >
> > > Is choose-ns the total number of ms used to choose a message from the
> > input
> > >> stream? What are some gating factors (e.g. serialization?) for this
> > >> metric?
> > >
> > > It's the amount of time the event loop spent getting new messsages for
> > > process(). It includes deserialization time and poll time which we
> added
> > > new metrics for, in Samza 10.1. Typically deserialization time is
> pretty
> > > consistent, so when you see a spike in choose-ns, it's usually because
> > the
> > > event loop is waiting for new messages. The two most common cases when
> > it's
> > > waiting are:
> > > 1. There are no new messages in the topic partition. This is good
> because
> > > it means the processor is caught up.
> > > 2. The consumer is slow and/or the buffer isn't large enough so the
> > > BrokerProxy isn't able to keep enough messages buffered to keep the
> event
> > > loop busy. This is uncommon because the buffer is defaulted to 50,000
> > > messages, which should be plenty. But if it happens, it's bad. To
> control
> > > this behavior, see the following properties in the config table (
> > > http://samza.apache.org/learn/documentation/0.10/jobs/
> > > configuration-table.html)
> > > systems.system-name.samza.fetch.threshold
> > > task.poll.interval.ms
> > >
> > >
> > >
> > > On Wed, Aug 24, 2016 at 8:52 AM, David Yu <da...@optimizely.com>
> > wrote:
> > >
> > >> More updates:
> > >> 1. process-envelopes rate finally stabilized and converged. Consumer
> lag
> > >> is
> > >> down to zero.
> > >> 2. avg choose-ns across containers dropped overtime
> > >> <https://www.dropbox.com/s/z4iiilvd7c1wrjc/Screenshot%202016
> > >> -08-24%2010.46.22.png?dl=0>,
> > >> which I assume is a good thing.
> > >>
> > >> My question:
> > >> Is choose-ns the total number of ms used to choose a message from the
> > >> input
> > >> stream? What are some gating factors (e.g. serialization?) for this
> > >> metric?
> > >>
> > >> Thanks,
> > >> David
> > >>
> > >> On Wed, Aug 24, 2016 at 12:34 AM David Yu <da...@optimizely.com>
> > >> wrote:
> > >>
> > >> > Some metric updates:
> > >> > 1. We started seeing some containers with a higher choose-ns
> > >> > <https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%202016
> > >> -08-24%2000.26.07.png?dl=0>.
> > >> > Not sure what would be the cause of this.
> > >> > 2. We are seeing very different process-envelopes values across
> > >> containers
> > >> > <https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%202016
> > >> -08-24%2000.21.05.png?dl=0>
> > >> > .
> > >> >
> > >> >
> > >> >
> > >> > On Tue, Aug 23, 2016 at 5:56 PM David Yu <da...@optimizely.com>
> > >> wrote:
> > >> >
> > >> >> Hi, Jake,
> > >> >>
> > >> >> Thanks for your suggestions. Some of my answers inline:
> > >> >>
> > >> >> 1.
> > >> >> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <ja...@gmail.com>
> > >> wrote:
> > >> >>
> > >> >>> Hey David,
> > >> >>>
> > >> >>> A few initial thoughts/questions:
> > >> >>>
> > >> >>>
> > >> >>>    1. Is this job using RocksDB to store the aggregations? If so,
> is
> > >> it
> > >> >>>    running on a machine with SSDs? We've seen a few performance
> > issues
> > >> >>> related
> > >> >>>    to RocksDB.
> > >> >>>       1. Not running on SSD causes slowness on disk
> > >> >>
> > >> >>  - [David] This definitely pointed me to the right direction in my
> > >> >> investigation. We do use RocksDB and just realized that our YARN
> > >> cluster is
> > >> >> using c3.xlarge EC2 instances and is using a mixture of EBS and
> local
> > >> SSD
> > >> >> storage. After digging around, we noticed that some containers were
> > >> >> persisting their KV stores in SSD while others were using EBS. We
> > just
> > >> >> updated our YARN config to use SSD only before redeployed our jobs.
> > So
> > >> far
> > >> >> everything looks good. Will report back on the performance update.
> > >> >>
> > >> >>>       2. Prior to Samza 10.1, samza would excessively flush the
> > store
> > >> to
> > >> >>>       disk, causing RocksDB compaction issues (stalls) - SAMZA-957
> > >> >>>
> > >> >> - [David] We did notice that the log cleaner thread died on one of
> > our
> > >> >> brokers. Not sure if this was the same problem you pointed out.
> > >> Following
> > >> >> errors are logged:
> > >> >>
> > >> >> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
> > >> >> [kafka-log-cleaner-thread-0], Error due to
> > >> >>
> > >> >> java.lang.IllegalArgumentException: requirement failed: 5865800
> > >> messages
> > >> >> in segment
> > session-store-2.0-tickets-changelog-9/00000000000009548937.
> > >> log
> > >> >> but offset map can fit only 5033164. You can increase
> > >> >> log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
> > >> >>
> > >> >>         at scala.Predef$.require(Predef.scala:219)
> > >> >>
> > >> >> We had to cleanup the changelog topic and restart the broker to
> bring
> > >> >> back the cleaner thread.
> > >> >>
> > >> >>>       3. When the RocksDB store is used as a queue, the iterator
> can
> > >> >>> suffer
> > >> >>>       performance issues due to RocksDBs tombstoning. (
> > >> >>>
> > >> >>> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Ser
> > >> vice-Using-RocksDB
> > >> >>>       )
> > >> >>>
> > >> >> - [David] We use RocksDB to keep track of opening sessions and use
> > >> >> sessionId (a random hash) as the key. In that sense, this does not
> > >> sound
> > >> >> like a queue. But we do iterate and delete closed sessions during
> > >> windowing
> > >> >> on a minute by minute basis.
> > >> >>
> > >> >>    2. Is the "messages-behind-high-watermark" metric non-zero?
> > >> >>>
> > >> >> -[David] Yes.
> > >> >>
> > >> >>>    3. The SamzaContainerMetrics might be useful too. Particularly
> > >> >>>    "choose-ns" and "commit-ns"
> > >> >>>
> > >> >> -[David] We are seeing the following from one of the containers
> > (after
> > >> >> the SSD fix mentioned above):
> > >> >> choose-ns=61353
> > >> >> commit-ns=306328 (what does this metric indicate? Is this in ms?)
> > >> >> process-ns=248260
> > >> >> window-ns=150717
> > >> >>
> > >> >>>    4. The only time I've personally seen slowness on the producer
> is
> > >> if
> > >> >>>    it's configured for acks="all". What is the producer config
> from
> > >> the
> > >> >>> log?
> > >> >>>
> > >> >> - [David] We did not override this. So should be the default value
> > >> (1?).
> > >> >>
> > >> >>    5. The window time is high, but since it's only called once per
> > >> minute,
> > >> >>>    it looks like it only represents 1% of the event loop
> > utilization.
> > >> So
> > >> >>> I
> > >> >>>    don't think that's a smoking gun.
> > >> >>>
> > >> >>> -Jake
> > >> >>>
> > >> >>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <
> david.yu@optimizely.com>
> > >> >>> wrote:
> > >> >>>
> > >> >>> > Dear Samza guys,
> > >> >>> >
> > >> >>> > We are here for some debugging suggestions on our Samza job
> > >> (0.10.0),
> > >> >>> which
> > >> >>> > lags behind on consumption after running for a couple of hours,
> > >> >>> regardless
> > >> >>> > of the number of containers allocated (currently 5).
> > >> >>> >
> > >> >>> > Briefly, the job aggregates events into sessions (in Avro)
> during
> > >> >>> process()
> > >> >>> > and emits snapshots of the open sessions using window() every
> > >> minute.
> > >> >>> This
> > >> >>> > graph
> > >> >>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
> > >> >>> > 202016-08-23%2010.33.16.png?dl=0>
> > >> >>> > shows
> > >> >>> > you where processing started to lag (red is the number of events
> > >> >>> received
> > >> >>> > and green is the number of event processed). The end result is a
> > >> steady
> > >> >>> > increase of the consumer lag
> > >> >>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
> > >> >>> > 202016-08-23%2010.19.27.png?dl=0>.
> > >> >>> > What we are trying to track down is where the performance
> > bottleneck
> > >> >>> is.
> > >> >>> > But it's unclear at the moment if that's in Samza or in Kafka.
> > >> >>> >
> > >> >>> > What we know so far:
> > >> >>> >
> > >> >>> >    - Kafka producer seems to take a while writing to the
> > downstream
> > >> >>> topic
> > >> >>> >    (changelog and session snapshots) shown by various timers.
> Not
> > >> sure
> > >> >>> > which
> > >> >>> >    numbers are critical but here are the producer metrics
> > >> >>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
> > >> >>> > 202016-08-23%2010.57.33.png?dl=0>
> > >> >>> > from
> > >> >>> >    one container.
> > >> >>> >    - avg windowing duration peaks at one point during the day
> (due
> > >> to
> > >> >>> the
> > >> >>> >    number of open sessions) but everything is still sub-seconds
> > >> >>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
> > >> >>> > 202016-08-23%2010.44.19.png?dl=0>
> > >> >>> >    .
> > >> >>> >    - our Kafka cluster doesn't seem to be overloaded
> > >> >>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
> > >> >>> > 202016-08-23%2010.48.25.png?dl=0>
> > >> >>> >     with writes < 60MB/s across all three brokers
> > >> >>> >
> > >> >>> > From all we know, we suspected that the bottleneck happens at
> > >> >>> producing to
> > >> >>> > Kafka. But we need some help confirming that.
> > >> >>> >
> > >> >>> > Any suggestion is appreciated.
> > >> >>> >
> > >> >>> > David
> > >> >>> >
> > >> >>>
> > >> >>
> > >>
> > >
> > >
> >
>

Re: Debug Samza consumer lag issue

Posted by David Yu <da...@optimizely.com>.
Great. It all makes sense now.

With the SSD fix, we also upgrade to 0.10.1. So we should see pretty
consistent process-ns (which we do).

Based on what you have described, the following should be true in 0.10.1:
event-loop-ns = choose-ns + process-ns + window-ns (if necessary) +
commit-ns (if necessary)

Is this correct?
Thanks,
David

On Wed, Aug 24, 2016 at 11:27 AM Jacob Maes <ja...@gmail.com> wrote:

> A couple other notes.
>
> Prior to Samza 10.1, the choose-ns was part of process-ns. So when
> choose-ns and process-ns are both high (around 10,000,000 == 10ms, which is
> the default poll timeout), that usually means the task is caught up. In
> Samza 10.1 the same is true if ONLY choose-ns is high. process-ns is always
> the time spent in the process() method.
>
> Based on the above, the metric numbers you provided after the SSD fix all
> look reasonable. They're all sub-millisecond and since choose-ns and
> process-ns are low, it seems that the container is chewing through messages
> at a good rate.
>
> So I would conclude that the SSD fix was probably the right one and it just
> took the job a while to catch up to the backlog of messages. Once it caught
> up, the choose-ns and process-ns increased, which is normal when the
> processor is waiting for new messages.
>
> -Jake
>
> On Wed, Aug 24, 2016 at 9:05 AM, Jacob Maes <ja...@gmail.com> wrote:
>
> > Hey David,
> >
> > Answering the most recent question first, since it's also the easiest.
> :-)
> >
> > Is choose-ns the total number of ms used to choose a message from the
> input
> >> stream? What are some gating factors (e.g. serialization?) for this
> >> metric?
> >
> > It's the amount of time the event loop spent getting new messsages for
> > process(). It includes deserialization time and poll time which we added
> > new metrics for, in Samza 10.1. Typically deserialization time is pretty
> > consistent, so when you see a spike in choose-ns, it's usually because
> the
> > event loop is waiting for new messages. The two most common cases when
> it's
> > waiting are:
> > 1. There are no new messages in the topic partition. This is good because
> > it means the processor is caught up.
> > 2. The consumer is slow and/or the buffer isn't large enough so the
> > BrokerProxy isn't able to keep enough messages buffered to keep the event
> > loop busy. This is uncommon because the buffer is defaulted to 50,000
> > messages, which should be plenty. But if it happens, it's bad. To control
> > this behavior, see the following properties in the config table (
> > http://samza.apache.org/learn/documentation/0.10/jobs/
> > configuration-table.html)
> > systems.system-name.samza.fetch.threshold
> > task.poll.interval.ms
> >
> >
> >
> > On Wed, Aug 24, 2016 at 8:52 AM, David Yu <da...@optimizely.com>
> wrote:
> >
> >> More updates:
> >> 1. process-envelopes rate finally stabilized and converged. Consumer lag
> >> is
> >> down to zero.
> >> 2. avg choose-ns across containers dropped overtime
> >> <https://www.dropbox.com/s/z4iiilvd7c1wrjc/Screenshot%202016
> >> -08-24%2010.46.22.png?dl=0>,
> >> which I assume is a good thing.
> >>
> >> My question:
> >> Is choose-ns the total number of ms used to choose a message from the
> >> input
> >> stream? What are some gating factors (e.g. serialization?) for this
> >> metric?
> >>
> >> Thanks,
> >> David
> >>
> >> On Wed, Aug 24, 2016 at 12:34 AM David Yu <da...@optimizely.com>
> >> wrote:
> >>
> >> > Some metric updates:
> >> > 1. We started seeing some containers with a higher choose-ns
> >> > <https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%202016
> >> -08-24%2000.26.07.png?dl=0>.
> >> > Not sure what would be the cause of this.
> >> > 2. We are seeing very different process-envelopes values across
> >> containers
> >> > <https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%202016
> >> -08-24%2000.21.05.png?dl=0>
> >> > .
> >> >
> >> >
> >> >
> >> > On Tue, Aug 23, 2016 at 5:56 PM David Yu <da...@optimizely.com>
> >> wrote:
> >> >
> >> >> Hi, Jake,
> >> >>
> >> >> Thanks for your suggestions. Some of my answers inline:
> >> >>
> >> >> 1.
> >> >> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <ja...@gmail.com>
> >> wrote:
> >> >>
> >> >>> Hey David,
> >> >>>
> >> >>> A few initial thoughts/questions:
> >> >>>
> >> >>>
> >> >>>    1. Is this job using RocksDB to store the aggregations? If so, is
> >> it
> >> >>>    running on a machine with SSDs? We've seen a few performance
> issues
> >> >>> related
> >> >>>    to RocksDB.
> >> >>>       1. Not running on SSD causes slowness on disk
> >> >>
> >> >>  - [David] This definitely pointed me to the right direction in my
> >> >> investigation. We do use RocksDB and just realized that our YARN
> >> cluster is
> >> >> using c3.xlarge EC2 instances and is using a mixture of EBS and local
> >> SSD
> >> >> storage. After digging around, we noticed that some containers were
> >> >> persisting their KV stores in SSD while others were using EBS. We
> just
> >> >> updated our YARN config to use SSD only before redeployed our jobs.
> So
> >> far
> >> >> everything looks good. Will report back on the performance update.
> >> >>
> >> >>>       2. Prior to Samza 10.1, samza would excessively flush the
> store
> >> to
> >> >>>       disk, causing RocksDB compaction issues (stalls) - SAMZA-957
> >> >>>
> >> >> - [David] We did notice that the log cleaner thread died on one of
> our
> >> >> brokers. Not sure if this was the same problem you pointed out.
> >> Following
> >> >> errors are logged:
> >> >>
> >> >> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
> >> >> [kafka-log-cleaner-thread-0], Error due to
> >> >>
> >> >> java.lang.IllegalArgumentException: requirement failed: 5865800
> >> messages
> >> >> in segment
> session-store-2.0-tickets-changelog-9/00000000000009548937.
> >> log
> >> >> but offset map can fit only 5033164. You can increase
> >> >> log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
> >> >>
> >> >>         at scala.Predef$.require(Predef.scala:219)
> >> >>
> >> >> We had to cleanup the changelog topic and restart the broker to bring
> >> >> back the cleaner thread.
> >> >>
> >> >>>       3. When the RocksDB store is used as a queue, the iterator can
> >> >>> suffer
> >> >>>       performance issues due to RocksDBs tombstoning. (
> >> >>>
> >> >>> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Ser
> >> vice-Using-RocksDB
> >> >>>       )
> >> >>>
> >> >> - [David] We use RocksDB to keep track of opening sessions and use
> >> >> sessionId (a random hash) as the key. In that sense, this does not
> >> sound
> >> >> like a queue. But we do iterate and delete closed sessions during
> >> windowing
> >> >> on a minute by minute basis.
> >> >>
> >> >>    2. Is the "messages-behind-high-watermark" metric non-zero?
> >> >>>
> >> >> -[David] Yes.
> >> >>
> >> >>>    3. The SamzaContainerMetrics might be useful too. Particularly
> >> >>>    "choose-ns" and "commit-ns"
> >> >>>
> >> >> -[David] We are seeing the following from one of the containers
> (after
> >> >> the SSD fix mentioned above):
> >> >> choose-ns=61353
> >> >> commit-ns=306328 (what does this metric indicate? Is this in ms?)
> >> >> process-ns=248260
> >> >> window-ns=150717
> >> >>
> >> >>>    4. The only time I've personally seen slowness on the producer is
> >> if
> >> >>>    it's configured for acks="all". What is the producer config from
> >> the
> >> >>> log?
> >> >>>
> >> >> - [David] We did not override this. So should be the default value
> >> (1?).
> >> >>
> >> >>    5. The window time is high, but since it's only called once per
> >> minute,
> >> >>>    it looks like it only represents 1% of the event loop
> utilization.
> >> So
> >> >>> I
> >> >>>    don't think that's a smoking gun.
> >> >>>
> >> >>> -Jake
> >> >>>
> >> >>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <da...@optimizely.com>
> >> >>> wrote:
> >> >>>
> >> >>> > Dear Samza guys,
> >> >>> >
> >> >>> > We are here for some debugging suggestions on our Samza job
> >> (0.10.0),
> >> >>> which
> >> >>> > lags behind on consumption after running for a couple of hours,
> >> >>> regardless
> >> >>> > of the number of containers allocated (currently 5).
> >> >>> >
> >> >>> > Briefly, the job aggregates events into sessions (in Avro) during
> >> >>> process()
> >> >>> > and emits snapshots of the open sessions using window() every
> >> minute.
> >> >>> This
> >> >>> > graph
> >> >>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
> >> >>> > 202016-08-23%2010.33.16.png?dl=0>
> >> >>> > shows
> >> >>> > you where processing started to lag (red is the number of events
> >> >>> received
> >> >>> > and green is the number of event processed). The end result is a
> >> steady
> >> >>> > increase of the consumer lag
> >> >>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
> >> >>> > 202016-08-23%2010.19.27.png?dl=0>.
> >> >>> > What we are trying to track down is where the performance
> bottleneck
> >> >>> is.
> >> >>> > But it's unclear at the moment if that's in Samza or in Kafka.
> >> >>> >
> >> >>> > What we know so far:
> >> >>> >
> >> >>> >    - Kafka producer seems to take a while writing to the
> downstream
> >> >>> topic
> >> >>> >    (changelog and session snapshots) shown by various timers. Not
> >> sure
> >> >>> > which
> >> >>> >    numbers are critical but here are the producer metrics
> >> >>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
> >> >>> > 202016-08-23%2010.57.33.png?dl=0>
> >> >>> > from
> >> >>> >    one container.
> >> >>> >    - avg windowing duration peaks at one point during the day (due
> >> to
> >> >>> the
> >> >>> >    number of open sessions) but everything is still sub-seconds
> >> >>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
> >> >>> > 202016-08-23%2010.44.19.png?dl=0>
> >> >>> >    .
> >> >>> >    - our Kafka cluster doesn't seem to be overloaded
> >> >>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
> >> >>> > 202016-08-23%2010.48.25.png?dl=0>
> >> >>> >     with writes < 60MB/s across all three brokers
> >> >>> >
> >> >>> > From all we know, we suspected that the bottleneck happens at
> >> >>> producing to
> >> >>> > Kafka. But we need some help confirming that.
> >> >>> >
> >> >>> > Any suggestion is appreciated.
> >> >>> >
> >> >>> > David
> >> >>> >
> >> >>>
> >> >>
> >>
> >
> >
>

Re: Debug Samza consumer lag issue

Posted by Jacob Maes <ja...@gmail.com>.
A couple other notes.

Prior to Samza 10.1, the choose-ns was part of process-ns. So when
choose-ns and process-ns are both high (around 10,000,000 == 10ms, which is
the default poll timeout), that usually means the task is caught up. In
Samza 10.1 the same is true if ONLY choose-ns is high. process-ns is always
the time spent in the process() method.

Based on the above, the metric numbers you provided after the SSD fix all
look reasonable. They're all sub-millisecond and since choose-ns and
process-ns are low, it seems that the container is chewing through messages
at a good rate.

So I would conclude that the SSD fix was probably the right one and it just
took the job a while to catch up to the backlog of messages. Once it caught
up, the choose-ns and process-ns increased, which is normal when the
processor is waiting for new messages.

-Jake

On Wed, Aug 24, 2016 at 9:05 AM, Jacob Maes <ja...@gmail.com> wrote:

> Hey David,
>
> Answering the most recent question first, since it's also the easiest. :-)
>
> Is choose-ns the total number of ms used to choose a message from the input
>> stream? What are some gating factors (e.g. serialization?) for this
>> metric?
>
> It's the amount of time the event loop spent getting new messsages for
> process(). It includes deserialization time and poll time which we added
> new metrics for, in Samza 10.1. Typically deserialization time is pretty
> consistent, so when you see a spike in choose-ns, it's usually because the
> event loop is waiting for new messages. The two most common cases when it's
> waiting are:
> 1. There are no new messages in the topic partition. This is good because
> it means the processor is caught up.
> 2. The consumer is slow and/or the buffer isn't large enough so the
> BrokerProxy isn't able to keep enough messages buffered to keep the event
> loop busy. This is uncommon because the buffer is defaulted to 50,000
> messages, which should be plenty. But if it happens, it's bad. To control
> this behavior, see the following properties in the config table (
> http://samza.apache.org/learn/documentation/0.10/jobs/
> configuration-table.html)
> systems.system-name.samza.fetch.threshold
> task.poll.interval.ms
>
>
>
> On Wed, Aug 24, 2016 at 8:52 AM, David Yu <da...@optimizely.com> wrote:
>
>> More updates:
>> 1. process-envelopes rate finally stabilized and converged. Consumer lag
>> is
>> down to zero.
>> 2. avg choose-ns across containers dropped overtime
>> <https://www.dropbox.com/s/z4iiilvd7c1wrjc/Screenshot%202016
>> -08-24%2010.46.22.png?dl=0>,
>> which I assume is a good thing.
>>
>> My question:
>> Is choose-ns the total number of ms used to choose a message from the
>> input
>> stream? What are some gating factors (e.g. serialization?) for this
>> metric?
>>
>> Thanks,
>> David
>>
>> On Wed, Aug 24, 2016 at 12:34 AM David Yu <da...@optimizely.com>
>> wrote:
>>
>> > Some metric updates:
>> > 1. We started seeing some containers with a higher choose-ns
>> > <https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%202016
>> -08-24%2000.26.07.png?dl=0>.
>> > Not sure what would be the cause of this.
>> > 2. We are seeing very different process-envelopes values across
>> containers
>> > <https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%202016
>> -08-24%2000.21.05.png?dl=0>
>> > .
>> >
>> >
>> >
>> > On Tue, Aug 23, 2016 at 5:56 PM David Yu <da...@optimizely.com>
>> wrote:
>> >
>> >> Hi, Jake,
>> >>
>> >> Thanks for your suggestions. Some of my answers inline:
>> >>
>> >> 1.
>> >> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <ja...@gmail.com>
>> wrote:
>> >>
>> >>> Hey David,
>> >>>
>> >>> A few initial thoughts/questions:
>> >>>
>> >>>
>> >>>    1. Is this job using RocksDB to store the aggregations? If so, is
>> it
>> >>>    running on a machine with SSDs? We've seen a few performance issues
>> >>> related
>> >>>    to RocksDB.
>> >>>       1. Not running on SSD causes slowness on disk
>> >>
>> >>  - [David] This definitely pointed me to the right direction in my
>> >> investigation. We do use RocksDB and just realized that our YARN
>> cluster is
>> >> using c3.xlarge EC2 instances and is using a mixture of EBS and local
>> SSD
>> >> storage. After digging around, we noticed that some containers were
>> >> persisting their KV stores in SSD while others were using EBS. We just
>> >> updated our YARN config to use SSD only before redeployed our jobs. So
>> far
>> >> everything looks good. Will report back on the performance update.
>> >>
>> >>>       2. Prior to Samza 10.1, samza would excessively flush the store
>> to
>> >>>       disk, causing RocksDB compaction issues (stalls) - SAMZA-957
>> >>>
>> >> - [David] We did notice that the log cleaner thread died on one of our
>> >> brokers. Not sure if this was the same problem you pointed out.
>> Following
>> >> errors are logged:
>> >>
>> >> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
>> >> [kafka-log-cleaner-thread-0], Error due to
>> >>
>> >> java.lang.IllegalArgumentException: requirement failed: 5865800
>> messages
>> >> in segment session-store-2.0-tickets-changelog-9/00000000000009548937.
>> log
>> >> but offset map can fit only 5033164. You can increase
>> >> log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
>> >>
>> >>         at scala.Predef$.require(Predef.scala:219)
>> >>
>> >> We had to cleanup the changelog topic and restart the broker to bring
>> >> back the cleaner thread.
>> >>
>> >>>       3. When the RocksDB store is used as a queue, the iterator can
>> >>> suffer
>> >>>       performance issues due to RocksDBs tombstoning. (
>> >>>
>> >>> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Ser
>> vice-Using-RocksDB
>> >>>       )
>> >>>
>> >> - [David] We use RocksDB to keep track of opening sessions and use
>> >> sessionId (a random hash) as the key. In that sense, this does not
>> sound
>> >> like a queue. But we do iterate and delete closed sessions during
>> windowing
>> >> on a minute by minute basis.
>> >>
>> >>    2. Is the "messages-behind-high-watermark" metric non-zero?
>> >>>
>> >> -[David] Yes.
>> >>
>> >>>    3. The SamzaContainerMetrics might be useful too. Particularly
>> >>>    "choose-ns" and "commit-ns"
>> >>>
>> >> -[David] We are seeing the following from one of the containers (after
>> >> the SSD fix mentioned above):
>> >> choose-ns=61353
>> >> commit-ns=306328 (what does this metric indicate? Is this in ms?)
>> >> process-ns=248260
>> >> window-ns=150717
>> >>
>> >>>    4. The only time I've personally seen slowness on the producer is
>> if
>> >>>    it's configured for acks="all". What is the producer config from
>> the
>> >>> log?
>> >>>
>> >> - [David] We did not override this. So should be the default value
>> (1?).
>> >>
>> >>    5. The window time is high, but since it's only called once per
>> minute,
>> >>>    it looks like it only represents 1% of the event loop utilization.
>> So
>> >>> I
>> >>>    don't think that's a smoking gun.
>> >>>
>> >>> -Jake
>> >>>
>> >>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <da...@optimizely.com>
>> >>> wrote:
>> >>>
>> >>> > Dear Samza guys,
>> >>> >
>> >>> > We are here for some debugging suggestions on our Samza job
>> (0.10.0),
>> >>> which
>> >>> > lags behind on consumption after running for a couple of hours,
>> >>> regardless
>> >>> > of the number of containers allocated (currently 5).
>> >>> >
>> >>> > Briefly, the job aggregates events into sessions (in Avro) during
>> >>> process()
>> >>> > and emits snapshots of the open sessions using window() every
>> minute.
>> >>> This
>> >>> > graph
>> >>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
>> >>> > 202016-08-23%2010.33.16.png?dl=0>
>> >>> > shows
>> >>> > you where processing started to lag (red is the number of events
>> >>> received
>> >>> > and green is the number of event processed). The end result is a
>> steady
>> >>> > increase of the consumer lag
>> >>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
>> >>> > 202016-08-23%2010.19.27.png?dl=0>.
>> >>> > What we are trying to track down is where the performance bottleneck
>> >>> is.
>> >>> > But it's unclear at the moment if that's in Samza or in Kafka.
>> >>> >
>> >>> > What we know so far:
>> >>> >
>> >>> >    - Kafka producer seems to take a while writing to the downstream
>> >>> topic
>> >>> >    (changelog and session snapshots) shown by various timers. Not
>> sure
>> >>> > which
>> >>> >    numbers are critical but here are the producer metrics
>> >>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
>> >>> > 202016-08-23%2010.57.33.png?dl=0>
>> >>> > from
>> >>> >    one container.
>> >>> >    - avg windowing duration peaks at one point during the day (due
>> to
>> >>> the
>> >>> >    number of open sessions) but everything is still sub-seconds
>> >>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
>> >>> > 202016-08-23%2010.44.19.png?dl=0>
>> >>> >    .
>> >>> >    - our Kafka cluster doesn't seem to be overloaded
>> >>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
>> >>> > 202016-08-23%2010.48.25.png?dl=0>
>> >>> >     with writes < 60MB/s across all three brokers
>> >>> >
>> >>> > From all we know, we suspected that the bottleneck happens at
>> >>> producing to
>> >>> > Kafka. But we need some help confirming that.
>> >>> >
>> >>> > Any suggestion is appreciated.
>> >>> >
>> >>> > David
>> >>> >
>> >>>
>> >>
>>
>
>

Re: Debug Samza consumer lag issue

Posted by Jacob Maes <ja...@gmail.com>.
Hey David,

Answering the most recent question first, since it's also the easiest. :-)

Is choose-ns the total number of ms used to choose a message from the input
> stream? What are some gating factors (e.g. serialization?) for this metric?

It's the amount of time the event loop spent getting new messsages for
process(). It includes deserialization time and poll time which we added
new metrics for, in Samza 10.1. Typically deserialization time is pretty
consistent, so when you see a spike in choose-ns, it's usually because the
event loop is waiting for new messages. The two most common cases when it's
waiting are:
1. There are no new messages in the topic partition. This is good because
it means the processor is caught up.
2. The consumer is slow and/or the buffer isn't large enough so the
BrokerProxy isn't able to keep enough messages buffered to keep the event
loop busy. This is uncommon because the buffer is defaulted to 50,000
messages, which should be plenty. But if it happens, it's bad. To control
this behavior, see the following properties in the config table (
http://samza.apache.org/learn/documentation/0.10/jobs/configuration-table.html
)
systems.system-name.samza.fetch.threshold
task.poll.interval.ms



On Wed, Aug 24, 2016 at 8:52 AM, David Yu <da...@optimizely.com> wrote:

> More updates:
> 1. process-envelopes rate finally stabilized and converged. Consumer lag is
> down to zero.
> 2. avg choose-ns across containers dropped overtime
> <https://www.dropbox.com/s/z4iiilvd7c1wrjc/Screenshot%
> 202016-08-24%2010.46.22.png?dl=0>,
> which I assume is a good thing.
>
> My question:
> Is choose-ns the total number of ms used to choose a message from the input
> stream? What are some gating factors (e.g. serialization?) for this metric?
>
> Thanks,
> David
>
> On Wed, Aug 24, 2016 at 12:34 AM David Yu <da...@optimizely.com> wrote:
>
> > Some metric updates:
> > 1. We started seeing some containers with a higher choose-ns
> > <https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%
> 202016-08-24%2000.26.07.png?dl=0>.
> > Not sure what would be the cause of this.
> > 2. We are seeing very different process-envelopes values across
> containers
> > <https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%
> 202016-08-24%2000.21.05.png?dl=0>
> > .
> >
> >
> >
> > On Tue, Aug 23, 2016 at 5:56 PM David Yu <da...@optimizely.com>
> wrote:
> >
> >> Hi, Jake,
> >>
> >> Thanks for your suggestions. Some of my answers inline:
> >>
> >> 1.
> >> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <ja...@gmail.com>
> wrote:
> >>
> >>> Hey David,
> >>>
> >>> A few initial thoughts/questions:
> >>>
> >>>
> >>>    1. Is this job using RocksDB to store the aggregations? If so, is it
> >>>    running on a machine with SSDs? We've seen a few performance issues
> >>> related
> >>>    to RocksDB.
> >>>       1. Not running on SSD causes slowness on disk
> >>
> >>  - [David] This definitely pointed me to the right direction in my
> >> investigation. We do use RocksDB and just realized that our YARN
> cluster is
> >> using c3.xlarge EC2 instances and is using a mixture of EBS and local
> SSD
> >> storage. After digging around, we noticed that some containers were
> >> persisting their KV stores in SSD while others were using EBS. We just
> >> updated our YARN config to use SSD only before redeployed our jobs. So
> far
> >> everything looks good. Will report back on the performance update.
> >>
> >>>       2. Prior to Samza 10.1, samza would excessively flush the store
> to
> >>>       disk, causing RocksDB compaction issues (stalls) - SAMZA-957
> >>>
> >> - [David] We did notice that the log cleaner thread died on one of our
> >> brokers. Not sure if this was the same problem you pointed out.
> Following
> >> errors are logged:
> >>
> >> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
> >> [kafka-log-cleaner-thread-0], Error due to
> >>
> >> java.lang.IllegalArgumentException: requirement failed: 5865800
> messages
> >> in segment session-store-2.0-tickets-changelog-9/
> 00000000000009548937.log
> >> but offset map can fit only 5033164. You can increase
> >> log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
> >>
> >>         at scala.Predef$.require(Predef.scala:219)
> >>
> >> We had to cleanup the changelog topic and restart the broker to bring
> >> back the cleaner thread.
> >>
> >>>       3. When the RocksDB store is used as a queue, the iterator can
> >>> suffer
> >>>       performance issues due to RocksDBs tombstoning. (
> >>>
> >>> https://github.com/facebook/rocksdb/wiki/Implement-Queue-
> Service-Using-RocksDB
> >>>       )
> >>>
> >> - [David] We use RocksDB to keep track of opening sessions and use
> >> sessionId (a random hash) as the key. In that sense, this does not sound
> >> like a queue. But we do iterate and delete closed sessions during
> windowing
> >> on a minute by minute basis.
> >>
> >>    2. Is the "messages-behind-high-watermark" metric non-zero?
> >>>
> >> -[David] Yes.
> >>
> >>>    3. The SamzaContainerMetrics might be useful too. Particularly
> >>>    "choose-ns" and "commit-ns"
> >>>
> >> -[David] We are seeing the following from one of the containers (after
> >> the SSD fix mentioned above):
> >> choose-ns=61353
> >> commit-ns=306328 (what does this metric indicate? Is this in ms?)
> >> process-ns=248260
> >> window-ns=150717
> >>
> >>>    4. The only time I've personally seen slowness on the producer is if
> >>>    it's configured for acks="all". What is the producer config from the
> >>> log?
> >>>
> >> - [David] We did not override this. So should be the default value (1?).
> >>
> >>    5. The window time is high, but since it's only called once per
> minute,
> >>>    it looks like it only represents 1% of the event loop utilization.
> So
> >>> I
> >>>    don't think that's a smoking gun.
> >>>
> >>> -Jake
> >>>
> >>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <da...@optimizely.com>
> >>> wrote:
> >>>
> >>> > Dear Samza guys,
> >>> >
> >>> > We are here for some debugging suggestions on our Samza job (0.10.0),
> >>> which
> >>> > lags behind on consumption after running for a couple of hours,
> >>> regardless
> >>> > of the number of containers allocated (currently 5).
> >>> >
> >>> > Briefly, the job aggregates events into sessions (in Avro) during
> >>> process()
> >>> > and emits snapshots of the open sessions using window() every minute.
> >>> This
> >>> > graph
> >>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
> >>> > 202016-08-23%2010.33.16.png?dl=0>
> >>> > shows
> >>> > you where processing started to lag (red is the number of events
> >>> received
> >>> > and green is the number of event processed). The end result is a
> steady
> >>> > increase of the consumer lag
> >>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
> >>> > 202016-08-23%2010.19.27.png?dl=0>.
> >>> > What we are trying to track down is where the performance bottleneck
> >>> is.
> >>> > But it's unclear at the moment if that's in Samza or in Kafka.
> >>> >
> >>> > What we know so far:
> >>> >
> >>> >    - Kafka producer seems to take a while writing to the downstream
> >>> topic
> >>> >    (changelog and session snapshots) shown by various timers. Not
> sure
> >>> > which
> >>> >    numbers are critical but here are the producer metrics
> >>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
> >>> > 202016-08-23%2010.57.33.png?dl=0>
> >>> > from
> >>> >    one container.
> >>> >    - avg windowing duration peaks at one point during the day (due to
> >>> the
> >>> >    number of open sessions) but everything is still sub-seconds
> >>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
> >>> > 202016-08-23%2010.44.19.png?dl=0>
> >>> >    .
> >>> >    - our Kafka cluster doesn't seem to be overloaded
> >>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
> >>> > 202016-08-23%2010.48.25.png?dl=0>
> >>> >     with writes < 60MB/s across all three brokers
> >>> >
> >>> > From all we know, we suspected that the bottleneck happens at
> >>> producing to
> >>> > Kafka. But we need some help confirming that.
> >>> >
> >>> > Any suggestion is appreciated.
> >>> >
> >>> > David
> >>> >
> >>>
> >>
>

Re: Debug Samza consumer lag issue

Posted by David Yu <da...@optimizely.com>.
More updates:
1. process-envelopes rate finally stabilized and converged. Consumer lag is
down to zero.
2. avg choose-ns across containers dropped overtime
<https://www.dropbox.com/s/z4iiilvd7c1wrjc/Screenshot%202016-08-24%2010.46.22.png?dl=0>,
which I assume is a good thing.

My question:
Is choose-ns the total number of ms used to choose a message from the input
stream? What are some gating factors (e.g. serialization?) for this metric?

Thanks,
David

On Wed, Aug 24, 2016 at 12:34 AM David Yu <da...@optimizely.com> wrote:

> Some metric updates:
> 1. We started seeing some containers with a higher choose-ns
> <https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%202016-08-24%2000.26.07.png?dl=0>.
> Not sure what would be the cause of this.
> 2. We are seeing very different process-envelopes values across containers
> <https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%202016-08-24%2000.21.05.png?dl=0>
> .
>
>
>
> On Tue, Aug 23, 2016 at 5:56 PM David Yu <da...@optimizely.com> wrote:
>
>> Hi, Jake,
>>
>> Thanks for your suggestions. Some of my answers inline:
>>
>> 1.
>> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <ja...@gmail.com> wrote:
>>
>>> Hey David,
>>>
>>> A few initial thoughts/questions:
>>>
>>>
>>>    1. Is this job using RocksDB to store the aggregations? If so, is it
>>>    running on a machine with SSDs? We've seen a few performance issues
>>> related
>>>    to RocksDB.
>>>       1. Not running on SSD causes slowness on disk
>>
>>  - [David] This definitely pointed me to the right direction in my
>> investigation. We do use RocksDB and just realized that our YARN cluster is
>> using c3.xlarge EC2 instances and is using a mixture of EBS and local SSD
>> storage. After digging around, we noticed that some containers were
>> persisting their KV stores in SSD while others were using EBS. We just
>> updated our YARN config to use SSD only before redeployed our jobs. So far
>> everything looks good. Will report back on the performance update.
>>
>>>       2. Prior to Samza 10.1, samza would excessively flush the store to
>>>       disk, causing RocksDB compaction issues (stalls) - SAMZA-957
>>>
>> - [David] We did notice that the log cleaner thread died on one of our
>> brokers. Not sure if this was the same problem you pointed out. Following
>> errors are logged:
>>
>> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
>> [kafka-log-cleaner-thread-0], Error due to
>>
>> java.lang.IllegalArgumentException: requirement failed: 5865800 messages
>> in segment session-store-2.0-tickets-changelog-9/00000000000009548937.log
>> but offset map can fit only 5033164. You can increase
>> log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
>>
>>         at scala.Predef$.require(Predef.scala:219)
>>
>> We had to cleanup the changelog topic and restart the broker to bring
>> back the cleaner thread.
>>
>>>       3. When the RocksDB store is used as a queue, the iterator can
>>> suffer
>>>       performance issues due to RocksDBs tombstoning. (
>>>
>>> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Service-Using-RocksDB
>>>       )
>>>
>> - [David] We use RocksDB to keep track of opening sessions and use
>> sessionId (a random hash) as the key. In that sense, this does not sound
>> like a queue. But we do iterate and delete closed sessions during windowing
>> on a minute by minute basis.
>>
>>    2. Is the "messages-behind-high-watermark" metric non-zero?
>>>
>> -[David] Yes.
>>
>>>    3. The SamzaContainerMetrics might be useful too. Particularly
>>>    "choose-ns" and "commit-ns"
>>>
>> -[David] We are seeing the following from one of the containers (after
>> the SSD fix mentioned above):
>> choose-ns=61353
>> commit-ns=306328 (what does this metric indicate? Is this in ms?)
>> process-ns=248260
>> window-ns=150717
>>
>>>    4. The only time I've personally seen slowness on the producer is if
>>>    it's configured for acks="all". What is the producer config from the
>>> log?
>>>
>> - [David] We did not override this. So should be the default value (1?).
>>
>>    5. The window time is high, but since it's only called once per minute,
>>>    it looks like it only represents 1% of the event loop utilization. So
>>> I
>>>    don't think that's a smoking gun.
>>>
>>> -Jake
>>>
>>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <da...@optimizely.com>
>>> wrote:
>>>
>>> > Dear Samza guys,
>>> >
>>> > We are here for some debugging suggestions on our Samza job (0.10.0),
>>> which
>>> > lags behind on consumption after running for a couple of hours,
>>> regardless
>>> > of the number of containers allocated (currently 5).
>>> >
>>> > Briefly, the job aggregates events into sessions (in Avro) during
>>> process()
>>> > and emits snapshots of the open sessions using window() every minute.
>>> This
>>> > graph
>>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
>>> > 202016-08-23%2010.33.16.png?dl=0>
>>> > shows
>>> > you where processing started to lag (red is the number of events
>>> received
>>> > and green is the number of event processed). The end result is a steady
>>> > increase of the consumer lag
>>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
>>> > 202016-08-23%2010.19.27.png?dl=0>.
>>> > What we are trying to track down is where the performance bottleneck
>>> is.
>>> > But it's unclear at the moment if that's in Samza or in Kafka.
>>> >
>>> > What we know so far:
>>> >
>>> >    - Kafka producer seems to take a while writing to the downstream
>>> topic
>>> >    (changelog and session snapshots) shown by various timers. Not sure
>>> > which
>>> >    numbers are critical but here are the producer metrics
>>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
>>> > 202016-08-23%2010.57.33.png?dl=0>
>>> > from
>>> >    one container.
>>> >    - avg windowing duration peaks at one point during the day (due to
>>> the
>>> >    number of open sessions) but everything is still sub-seconds
>>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
>>> > 202016-08-23%2010.44.19.png?dl=0>
>>> >    .
>>> >    - our Kafka cluster doesn't seem to be overloaded
>>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
>>> > 202016-08-23%2010.48.25.png?dl=0>
>>> >     with writes < 60MB/s across all three brokers
>>> >
>>> > From all we know, we suspected that the bottleneck happens at
>>> producing to
>>> > Kafka. But we need some help confirming that.
>>> >
>>> > Any suggestion is appreciated.
>>> >
>>> > David
>>> >
>>>
>>

Re: Debug Samza consumer lag issue

Posted by David Yu <da...@optimizely.com>.
Some metric updates:
1. We started seeing some containers with a higher choose-ns
<https://www.dropbox.com/s/06n3awdwn8ntfxd/Screenshot%202016-08-24%2000.26.07.png?dl=0>.
Not sure what would be the cause of this.
2. We are seeing very different process-envelopes values across containers
<https://www.dropbox.com/s/n1wxtngquv607nb/Screenshot%202016-08-24%2000.21.05.png?dl=0>
.



On Tue, Aug 23, 2016 at 5:56 PM David Yu <da...@optimizely.com> wrote:

> Hi, Jake,
>
> Thanks for your suggestions. Some of my answers inline:
>
> 1.
> On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <ja...@gmail.com> wrote:
>
>> Hey David,
>>
>> A few initial thoughts/questions:
>>
>>
>>    1. Is this job using RocksDB to store the aggregations? If so, is it
>>    running on a machine with SSDs? We've seen a few performance issues
>> related
>>    to RocksDB.
>>       1. Not running on SSD causes slowness on disk
>
>  - [David] This definitely pointed me to the right direction in my
> investigation. We do use RocksDB and just realized that our YARN cluster is
> using c3.xlarge EC2 instances and is using a mixture of EBS and local SSD
> storage. After digging around, we noticed that some containers were
> persisting their KV stores in SSD while others were using EBS. We just
> updated our YARN config to use SSD only before redeployed our jobs. So far
> everything looks good. Will report back on the performance update.
>
>>       2. Prior to Samza 10.1, samza would excessively flush the store to
>>       disk, causing RocksDB compaction issues (stalls) - SAMZA-957
>>
> - [David] We did notice that the log cleaner thread died on one of our
> brokers. Not sure if this was the same problem you pointed out. Following
> errors are logged:
>
> 2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
> [kafka-log-cleaner-thread-0], Error due to
>
> java.lang.IllegalArgumentException: requirement failed: 5865800 messages
> in segment session-store-2.0-tickets-changelog-9/00000000000009548937.log
> but offset map can fit only 5033164. You can increase
> log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
>
>         at scala.Predef$.require(Predef.scala:219)
>
> We had to cleanup the changelog topic and restart the broker to bring back
> the cleaner thread.
>
>>       3. When the RocksDB store is used as a queue, the iterator can
>> suffer
>>       performance issues due to RocksDBs tombstoning. (
>>
>> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Service-Using-RocksDB
>>       )
>>
> - [David] We use RocksDB to keep track of opening sessions and use
> sessionId (a random hash) as the key. In that sense, this does not sound
> like a queue. But we do iterate and delete closed sessions during windowing
> on a minute by minute basis.
>
>    2. Is the "messages-behind-high-watermark" metric non-zero?
>>
> -[David] Yes.
>
>>    3. The SamzaContainerMetrics might be useful too. Particularly
>>    "choose-ns" and "commit-ns"
>>
> -[David] We are seeing the following from one of the containers (after the
> SSD fix mentioned above):
> choose-ns=61353
> commit-ns=306328 (what does this metric indicate? Is this in ms?)
> process-ns=248260
> window-ns=150717
>
>>    4. The only time I've personally seen slowness on the producer is if
>>    it's configured for acks="all". What is the producer config from the
>> log?
>>
> - [David] We did not override this. So should be the default value (1?).
>
>    5. The window time is high, but since it's only called once per minute,
>>    it looks like it only represents 1% of the event loop utilization. So I
>>    don't think that's a smoking gun.
>>
>> -Jake
>>
>> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <da...@optimizely.com>
>> wrote:
>>
>> > Dear Samza guys,
>> >
>> > We are here for some debugging suggestions on our Samza job (0.10.0),
>> which
>> > lags behind on consumption after running for a couple of hours,
>> regardless
>> > of the number of containers allocated (currently 5).
>> >
>> > Briefly, the job aggregates events into sessions (in Avro) during
>> process()
>> > and emits snapshots of the open sessions using window() every minute.
>> This
>> > graph
>> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
>> > 202016-08-23%2010.33.16.png?dl=0>
>> > shows
>> > you where processing started to lag (red is the number of events
>> received
>> > and green is the number of event processed). The end result is a steady
>> > increase of the consumer lag
>> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
>> > 202016-08-23%2010.19.27.png?dl=0>.
>> > What we are trying to track down is where the performance bottleneck is.
>> > But it's unclear at the moment if that's in Samza or in Kafka.
>> >
>> > What we know so far:
>> >
>> >    - Kafka producer seems to take a while writing to the downstream
>> topic
>> >    (changelog and session snapshots) shown by various timers. Not sure
>> > which
>> >    numbers are critical but here are the producer metrics
>> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
>> > 202016-08-23%2010.57.33.png?dl=0>
>> > from
>> >    one container.
>> >    - avg windowing duration peaks at one point during the day (due to
>> the
>> >    number of open sessions) but everything is still sub-seconds
>> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
>> > 202016-08-23%2010.44.19.png?dl=0>
>> >    .
>> >    - our Kafka cluster doesn't seem to be overloaded
>> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
>> > 202016-08-23%2010.48.25.png?dl=0>
>> >     with writes < 60MB/s across all three brokers
>> >
>> > From all we know, we suspected that the bottleneck happens at producing
>> to
>> > Kafka. But we need some help confirming that.
>> >
>> > Any suggestion is appreciated.
>> >
>> > David
>> >
>>
>

Re: Debug Samza consumer lag issue

Posted by David Yu <da...@optimizely.com>.
Hi, Jake,

Thanks for your suggestions. Some of my answers inline:

1.
On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <ja...@gmail.com> wrote:

> Hey David,
>
> A few initial thoughts/questions:
>
>
>    1. Is this job using RocksDB to store the aggregations? If so, is it
>    running on a machine with SSDs? We've seen a few performance issues
> related
>    to RocksDB.
>       1. Not running on SSD causes slowness on disk

 - [David] This definitely pointed me to the right direction in my
investigation. We do use RocksDB and just realized that our YARN cluster is
using c3.xlarge EC2 instances and is using a mixture of EBS and local SSD
storage. After digging around, we noticed that some containers were
persisting their KV stores in SSD while others were using EBS. We just
updated our YARN config to use SSD only before redeployed our jobs. So far
everything looks good. Will report back on the performance update.

>       2. Prior to Samza 10.1, samza would excessively flush the store to
>       disk, causing RocksDB compaction issues (stalls) - SAMZA-957
>
- [David] We did notice that the log cleaner thread died on one of our
brokers. Not sure if this was the same problem you pointed out. Following
errors are logged:

2016-08-15 10:00:56,475 ERROR kafka.log.LogCleaner:
[kafka-log-cleaner-thread-0], Error due to

java.lang.IllegalArgumentException: requirement failed: 5865800 messages in
segment session-store-2.0-tickets-changelog-9/00000000000009548937.log but
offset map can fit only 5033164. You can increase
log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads

        at scala.Predef$.require(Predef.scala:219)

We had to cleanup the changelog topic and restart the broker to bring back
the cleaner thread.

>       3. When the RocksDB store is used as a queue, the iterator can suffer
>       performance issues due to RocksDBs tombstoning. (
>
> https://github.com/facebook/rocksdb/wiki/Implement-Queue-Service-Using-RocksDB
>       )
>
- [David] We use RocksDB to keep track of opening sessions and use
sessionId (a random hash) as the key. In that sense, this does not sound
like a queue. But we do iterate and delete closed sessions during windowing
on a minute by minute basis.

   2. Is the "messages-behind-high-watermark" metric non-zero?
>
-[David] Yes.

>    3. The SamzaContainerMetrics might be useful too. Particularly
>    "choose-ns" and "commit-ns"
>
-[David] We are seeing the following from one of the containers (after the
SSD fix mentioned above):
choose-ns=61353
commit-ns=306328 (what does this metric indicate? Is this in ms?)
process-ns=248260
window-ns=150717

>    4. The only time I've personally seen slowness on the producer is if
>    it's configured for acks="all". What is the producer config from the
> log?
>
- [David] We did not override this. So should be the default value (1?).

   5. The window time is high, but since it's only called once per minute,
>    it looks like it only represents 1% of the event loop utilization. So I
>    don't think that's a smoking gun.
>
> -Jake
>
> On Tue, Aug 23, 2016 at 9:18 AM, David Yu <da...@optimizely.com> wrote:
>
> > Dear Samza guys,
> >
> > We are here for some debugging suggestions on our Samza job (0.10.0),
> which
> > lags behind on consumption after running for a couple of hours,
> regardless
> > of the number of containers allocated (currently 5).
> >
> > Briefly, the job aggregates events into sessions (in Avro) during
> process()
> > and emits snapshots of the open sessions using window() every minute.
> This
> > graph
> > <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
> > 202016-08-23%2010.33.16.png?dl=0>
> > shows
> > you where processing started to lag (red is the number of events received
> > and green is the number of event processed). The end result is a steady
> > increase of the consumer lag
> > <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
> > 202016-08-23%2010.19.27.png?dl=0>.
> > What we are trying to track down is where the performance bottleneck is.
> > But it's unclear at the moment if that's in Samza or in Kafka.
> >
> > What we know so far:
> >
> >    - Kafka producer seems to take a while writing to the downstream topic
> >    (changelog and session snapshots) shown by various timers. Not sure
> > which
> >    numbers are critical but here are the producer metrics
> >    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
> > 202016-08-23%2010.57.33.png?dl=0>
> > from
> >    one container.
> >    - avg windowing duration peaks at one point during the day (due to the
> >    number of open sessions) but everything is still sub-seconds
> >    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
> > 202016-08-23%2010.44.19.png?dl=0>
> >    .
> >    - our Kafka cluster doesn't seem to be overloaded
> >    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
> > 202016-08-23%2010.48.25.png?dl=0>
> >     with writes < 60MB/s across all three brokers
> >
> > From all we know, we suspected that the bottleneck happens at producing
> to
> > Kafka. But we need some help confirming that.
> >
> > Any suggestion is appreciated.
> >
> > David
> >
>

Re: Debug Samza consumer lag issue

Posted by Jacob Maes <ja...@gmail.com>.
Hey David,

A few initial thoughts/questions:


   1. Is this job using RocksDB to store the aggregations? If so, is it
   running on a machine with SSDs? We've seen a few performance issues related
   to RocksDB.
      1. Not running on SSD causes slowness on disk
      2. Prior to Samza 10.1, samza would excessively flush the store to
      disk, causing RocksDB compaction issues (stalls) - SAMZA-957
      3. When the RocksDB store is used as a queue, the iterator can suffer
      performance issues due to RocksDBs tombstoning. (
      https://github.com/facebook/rocksdb/wiki/Implement-Queue-Service-Using-RocksDB
      )
   2. Is the "messages-behind-high-watermark" metric non-zero?
   3. The SamzaContainerMetrics might be useful too. Particularly
   "choose-ns" and "commit-ns"
   4. The only time I've personally seen slowness on the producer is if
   it's configured for acks="all". What is the producer config from the log?
   5. The window time is high, but since it's only called once per minute,
   it looks like it only represents 1% of the event loop utilization. So I
   don't think that's a smoking gun.

-Jake

On Tue, Aug 23, 2016 at 9:18 AM, David Yu <da...@optimizely.com> wrote:

> Dear Samza guys,
>
> We are here for some debugging suggestions on our Samza job (0.10.0), which
> lags behind on consumption after running for a couple of hours, regardless
> of the number of containers allocated (currently 5).
>
> Briefly, the job aggregates events into sessions (in Avro) during process()
> and emits snapshots of the open sessions using window() every minute. This
> graph
> <https://www.dropbox.com/s/utywr1j5eku0ec0/Screenshot%
> 202016-08-23%2010.33.16.png?dl=0>
> shows
> you where processing started to lag (red is the number of events received
> and green is the number of event processed). The end result is a steady
> increase of the consumer lag
> <https://www.dropbox.com/s/fppsv91c339xmdb/Screenshot%
> 202016-08-23%2010.19.27.png?dl=0>.
> What we are trying to track down is where the performance bottleneck is.
> But it's unclear at the moment if that's in Samza or in Kafka.
>
> What we know so far:
>
>    - Kafka producer seems to take a while writing to the downstream topic
>    (changelog and session snapshots) shown by various timers. Not sure
> which
>    numbers are critical but here are the producer metrics
>    <https://www.dropbox.com/s/pzi9304gw5vmae2/Screenshot%
> 202016-08-23%2010.57.33.png?dl=0>
> from
>    one container.
>    - avg windowing duration peaks at one point during the day (due to the
>    number of open sessions) but everything is still sub-seconds
>    <https://www.dropbox.com/s/y2ps6pbs1tf257e/Screenshot%
> 202016-08-23%2010.44.19.png?dl=0>
>    .
>    - our Kafka cluster doesn't seem to be overloaded
>    <https://www.dropbox.com/s/q01b4p4rg43spua/Screenshot%
> 202016-08-23%2010.48.25.png?dl=0>
>     with writes < 60MB/s across all three brokers
>
> From all we know, we suspected that the bottleneck happens at producing to
> Kafka. But we need some help confirming that.
>
> Any suggestion is appreciated.
>
> David
>