Hi, Jake,

Thanks for your suggestions. Some of my answers inline:

1.
On Tue, Aug 23, 2016 at 11:53 AM Jacob Maes <jacob.m...@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 <david...@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
> >
>

Reply via email to