You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samza.apache.org by Karthik Sriram <am...@gmail.com> on 2015/02/10 01:35:54 UTC

Collocating Samza(YARN) and Kafka/ZK clusters

Hey all,
 I'm trying to run samza on a 5 node (YARN/Kafka/ZK) cluster with each box
running all 3 processes on AWS. I have been facing very weird performance
issues with Kafka when run this way. Kafka seems to get unbalanced very
often with replicas going out of sync every so often. This results in lost
messages when producing to this cluster. I initially suspected it was a
scale issue (70k-80k qps of incoming messages, ~120k qps peak) and reduced
write throughput by sampling just 10% of the messages but I still noticed
the same issues. The weird part is that this doesn't happen every time I
run, but many of the times.

We have been using a much larger Kafka cluster for long with great
performance and have never seen such issues before. Then I saw (
https://engineering.linkedin.com/samza/operating-apache-samza-scale) which
mentions that LinkedIn also faced some issues when collocating Samza and
Kafka.

Can someone throw some light on this? Is collocating samza and kafka a
strict no, or is it more likely a Kafka/machine tuning issue ? Any help is
appreciated!

Kafka version : 0.8.1.1
Samza version: 0.8

Thanks a lot for your time,
Karthik

Re: Collocating Samza(YARN) and Kafka/ZK clusters

Posted by Karthik Sriram <am...@gmail.com>.
Hey Jay/Chris,
  Thanks for your valuable input. I tried tuning the replica sync
timeout/ZK session timeout etc but it helped to keep the cluster stable a
little bit longer but not by much. I then stumbled upon and found that I
was probably hit by this https://issues.apache.org/jira/browse/KAFKA-1382.
We have been using Kafka 0.8.1.1 for our central logging system for a while
now without seeing much issues. It seems that we hit this issue only with
our smaller samza-kafka cluster that we have been pushing to the limits in
our testing phase. Or its possible that collocating ZK/YARN and Kafka
triggers this issue easily. We upgraded to Kafka 0.8.2 for the samza-kafka
cluster and everything works like a charm now. Thanks a lot for your input,
it gave me direction!

Thanks a lot,
Karthik


Thanks,
Karthik

On Tue, Feb 10, 2015 at 9:51 AM, Chris Riccomini <cr...@apache.org>
wrote:

> Hey Karthik,
>
> I've never tried running ZK on the same machines as Kafka/Samza.
>
> Co-locating Kafka/Samza worked pretty well for us until we started using
> Samza's state management facilities. At this point, Samza's state stores
> started messing with the OS page cache in a way that impacted the Kafka
> brokers' performance. Kafka doesn't really have a cache; it just uses page
> cache. So, when the page cache is being used for other things (e.g. RocksDB
> bytes), it causes Kafka to go to disk more often, which increases latency
> amongst consumers.
>
> If you're not running state with your Samza jobs, then it doesn't seem like
> the jobs should impact Kafka, unless you're over-provisioning the machines,
> and saturating the CPU or network.
>
> In general, it's probably a best practice not to run the jobs on the same
> machines as the brokers.
>
> Cheers,
> Chris
>
> On Mon, Feb 9, 2015 at 9:20 PM, Vijay Gill <vi...@gmail.com> wrote:
>
> > Is there a substantial variance in performance caused by high cpu load
> and
> > cache churn? I've seen this sort of inadequate perf isolation wreak havoc
> > on high QPS systems.
> >
> > On Mon Feb 09 2015 at 4:55:28 PM Jay Kreps <ja...@gmail.com> wrote:
> >
> > > It may or may not be due to colocating Kafka and Samza but you are
> > probably
> > > tripping the failure detection in Kafka which considers a replica out
> of
> > > sync if it falls more than N messages behind. Can you try tuning this
> > > setting as described here:
> > > https://cwiki.apache.org/confluence/display/KAFKA/FAQ#
> > > FAQ-HowtoreducechurnsinISR?WhendoesabrokerleavetheISR
> > > ?
> > >
> > > -Jay
> > >
> > > On Mon, Feb 9, 2015 at 4:35 PM, Karthik Sriram <am...@gmail.com>
> > > wrote:
> > >
> > > > Hey all,
> > > >  I'm trying to run samza on a 5 node (YARN/Kafka/ZK) cluster with
> each
> > > box
> > > > running all 3 processes on AWS. I have been facing very weird
> > performance
> > > > issues with Kafka when run this way. Kafka seems to get unbalanced
> very
> > > > often with replicas going out of sync every so often. This results in
> > > lost
> > > > messages when producing to this cluster. I initially suspected it
> was a
> > > > scale issue (70k-80k qps of incoming messages, ~120k qps peak) and
> > > reduced
> > > > write throughput by sampling just 10% of the messages but I still
> > noticed
> > > > the same issues. The weird part is that this doesn't happen every
> time
> > I
> > > > run, but many of the times.
> > > >
> > > > We have been using a much larger Kafka cluster for long with great
> > > > performance and have never seen such issues before. Then I saw (
> > > > https://engineering.linkedin.com/samza/operating-apache-samza-scale)
> > > which
> > > > mentions that LinkedIn also faced some issues when collocating Samza
> > and
> > > > Kafka.
> > > >
> > > > Can someone throw some light on this? Is collocating samza and kafka
> a
> > > > strict no, or is it more likely a Kafka/machine tuning issue ? Any
> help
> > > is
> > > > appreciated!
> > > >
> > > > Kafka version : 0.8.1.1
> > > > Samza version: 0.8
> > > >
> > > > Thanks a lot for your time,
> > > > Karthik
> > > >
> > >
> >
>

Re: Collocating Samza(YARN) and Kafka/ZK clusters

Posted by Chris Riccomini <cr...@apache.org>.
Hey Karthik,

I've never tried running ZK on the same machines as Kafka/Samza.

Co-locating Kafka/Samza worked pretty well for us until we started using
Samza's state management facilities. At this point, Samza's state stores
started messing with the OS page cache in a way that impacted the Kafka
brokers' performance. Kafka doesn't really have a cache; it just uses page
cache. So, when the page cache is being used for other things (e.g. RocksDB
bytes), it causes Kafka to go to disk more often, which increases latency
amongst consumers.

If you're not running state with your Samza jobs, then it doesn't seem like
the jobs should impact Kafka, unless you're over-provisioning the machines,
and saturating the CPU or network.

In general, it's probably a best practice not to run the jobs on the same
machines as the brokers.

Cheers,
Chris

On Mon, Feb 9, 2015 at 9:20 PM, Vijay Gill <vi...@gmail.com> wrote:

> Is there a substantial variance in performance caused by high cpu load and
> cache churn? I've seen this sort of inadequate perf isolation wreak havoc
> on high QPS systems.
>
> On Mon Feb 09 2015 at 4:55:28 PM Jay Kreps <ja...@gmail.com> wrote:
>
> > It may or may not be due to colocating Kafka and Samza but you are
> probably
> > tripping the failure detection in Kafka which considers a replica out of
> > sync if it falls more than N messages behind. Can you try tuning this
> > setting as described here:
> > https://cwiki.apache.org/confluence/display/KAFKA/FAQ#
> > FAQ-HowtoreducechurnsinISR?WhendoesabrokerleavetheISR
> > ?
> >
> > -Jay
> >
> > On Mon, Feb 9, 2015 at 4:35 PM, Karthik Sriram <am...@gmail.com>
> > wrote:
> >
> > > Hey all,
> > >  I'm trying to run samza on a 5 node (YARN/Kafka/ZK) cluster with each
> > box
> > > running all 3 processes on AWS. I have been facing very weird
> performance
> > > issues with Kafka when run this way. Kafka seems to get unbalanced very
> > > often with replicas going out of sync every so often. This results in
> > lost
> > > messages when producing to this cluster. I initially suspected it was a
> > > scale issue (70k-80k qps of incoming messages, ~120k qps peak) and
> > reduced
> > > write throughput by sampling just 10% of the messages but I still
> noticed
> > > the same issues. The weird part is that this doesn't happen every time
> I
> > > run, but many of the times.
> > >
> > > We have been using a much larger Kafka cluster for long with great
> > > performance and have never seen such issues before. Then I saw (
> > > https://engineering.linkedin.com/samza/operating-apache-samza-scale)
> > which
> > > mentions that LinkedIn also faced some issues when collocating Samza
> and
> > > Kafka.
> > >
> > > Can someone throw some light on this? Is collocating samza and kafka a
> > > strict no, or is it more likely a Kafka/machine tuning issue ? Any help
> > is
> > > appreciated!
> > >
> > > Kafka version : 0.8.1.1
> > > Samza version: 0.8
> > >
> > > Thanks a lot for your time,
> > > Karthik
> > >
> >
>

Re: Collocating Samza(YARN) and Kafka/ZK clusters

Posted by Vijay Gill <vi...@gmail.com>.
Is there a substantial variance in performance caused by high cpu load and
cache churn? I've seen this sort of inadequate perf isolation wreak havoc
on high QPS systems.

On Mon Feb 09 2015 at 4:55:28 PM Jay Kreps <ja...@gmail.com> wrote:

> It may or may not be due to colocating Kafka and Samza but you are probably
> tripping the failure detection in Kafka which considers a replica out of
> sync if it falls more than N messages behind. Can you try tuning this
> setting as described here:
> https://cwiki.apache.org/confluence/display/KAFKA/FAQ#
> FAQ-HowtoreducechurnsinISR?WhendoesabrokerleavetheISR
> ?
>
> -Jay
>
> On Mon, Feb 9, 2015 at 4:35 PM, Karthik Sriram <am...@gmail.com>
> wrote:
>
> > Hey all,
> >  I'm trying to run samza on a 5 node (YARN/Kafka/ZK) cluster with each
> box
> > running all 3 processes on AWS. I have been facing very weird performance
> > issues with Kafka when run this way. Kafka seems to get unbalanced very
> > often with replicas going out of sync every so often. This results in
> lost
> > messages when producing to this cluster. I initially suspected it was a
> > scale issue (70k-80k qps of incoming messages, ~120k qps peak) and
> reduced
> > write throughput by sampling just 10% of the messages but I still noticed
> > the same issues. The weird part is that this doesn't happen every time I
> > run, but many of the times.
> >
> > We have been using a much larger Kafka cluster for long with great
> > performance and have never seen such issues before. Then I saw (
> > https://engineering.linkedin.com/samza/operating-apache-samza-scale)
> which
> > mentions that LinkedIn also faced some issues when collocating Samza and
> > Kafka.
> >
> > Can someone throw some light on this? Is collocating samza and kafka a
> > strict no, or is it more likely a Kafka/machine tuning issue ? Any help
> is
> > appreciated!
> >
> > Kafka version : 0.8.1.1
> > Samza version: 0.8
> >
> > Thanks a lot for your time,
> > Karthik
> >
>

Re: Collocating Samza(YARN) and Kafka/ZK clusters

Posted by Jay Kreps <ja...@gmail.com>.
It may or may not be due to colocating Kafka and Samza but you are probably
tripping the failure detection in Kafka which considers a replica out of
sync if it falls more than N messages behind. Can you try tuning this
setting as described here:
https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-HowtoreducechurnsinISR?WhendoesabrokerleavetheISR
?

-Jay

On Mon, Feb 9, 2015 at 4:35 PM, Karthik Sriram <am...@gmail.com> wrote:

> Hey all,
>  I'm trying to run samza on a 5 node (YARN/Kafka/ZK) cluster with each box
> running all 3 processes on AWS. I have been facing very weird performance
> issues with Kafka when run this way. Kafka seems to get unbalanced very
> often with replicas going out of sync every so often. This results in lost
> messages when producing to this cluster. I initially suspected it was a
> scale issue (70k-80k qps of incoming messages, ~120k qps peak) and reduced
> write throughput by sampling just 10% of the messages but I still noticed
> the same issues. The weird part is that this doesn't happen every time I
> run, but many of the times.
>
> We have been using a much larger Kafka cluster for long with great
> performance and have never seen such issues before. Then I saw (
> https://engineering.linkedin.com/samza/operating-apache-samza-scale) which
> mentions that LinkedIn also faced some issues when collocating Samza and
> Kafka.
>
> Can someone throw some light on this? Is collocating samza and kafka a
> strict no, or is it more likely a Kafka/machine tuning issue ? Any help is
> appreciated!
>
> Kafka version : 0.8.1.1
> Samza version: 0.8
>
> Thanks a lot for your time,
> Karthik
>