Have you found what the actual bottleneck is? Is it the network send? Of
course this would be highly influenced by the brokers' performance. After
removing all compression work from the brokers we were able to get enough
throughput from them that it's not really a concern.

Another rough side-effect of the single synchronous send thread is that a
single degrading or otherwise slow broker can back up the producing for the
whole app. I haven't heard a great solution to this but would love to if
someone's come up with it.

-Chris



On Wed, Jan 1, 2014 at 9:10 AM, Gerrit Jansen van Vuuren <
gerrit...@gmail.com> wrote:

> I've seen this bottle neck regardless of using compression or not, bpth
> situations give me poor performance on sending to kafka via the scala
> producer api.
> On 1 Jan 2014 16:42, "Chris Hogue" <csho...@gmail.com> wrote:
>
> > Hi.
> >
> > When writing that blog we were using Kafka 0.7 as well. Understanding
> that
> > it probably wasn't the primary design goal, the separate send threads per
> > broker that offered a separation of compression were a convenient
> > side-effect of that design.
> >
> > We've since built new systems on 0.8 that have concentrated high
> throughput
> > on a small number of producers and had this discovery early on as well.
> >
> > Instead we've taken responsibility for the compression before the
> producer
> > and done that on separate threads as appropriate. While helpful for
> > compression on the producer application the main reason for this is to
> > prevent the broker from uncompressing and re-compressing each message as
> it
> > assigns offsets. There's a significant throughput advantage in doing
> this.
> >
> > Truthfully since switching to snappy the compression throughput on the
> > producer is much less of a concern in the overall context of the
> > application.
> >
> > There was some discussion of these issues in the 'Client Improvement
> > Discussion' thread a while ago where Jay provided some insight and
> > discussion on future directions.
> >
> > -Chris
> >
> >
> >
> >
> > On Wed, Jan 1, 2014 at 5:42 AM, yosi botzer <yosi.bot...@gmail.com>
> wrote:
> >
> > > This is very interesting, this is what I see as well. I wish someone
> > could
> > > explain why it is not as explained here:
> > > http://engineering.gnip.com/kafka-async-producer/
> > >
> > >
> > > On Wed, Jan 1, 2014 at 2:39 PM, Gerrit Jansen van Vuuren <
> > > gerrit...@gmail.com> wrote:
> > >
> > > > I don't know the code enough to comment on that (maybe someone else
> on
> > > the
> > > > user list can do that), but from what I've seen doing some heavy
> > > profiling
> > > > I only see one thread per producer instance, it doesn't matter how
> many
> > > > brokers or topics you have the number of threads is always 1 per
> > > producer.
> > > > If you create 2 producers 2 threads and so on.
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On Wed, Jan 1, 2014 at 1:27 PM, yosi botzer <yosi.bot...@gmail.com>
> > > wrote:
> > > >
> > > > > But shouldn't I see a separate thread per broker (I am using the
> > async
> > > > > mode)?  Why do I get a better performance sending a message that
> has
> > > > fewer
> > > > > partitions?
> > > > >
> > > > >
> > > > > On Wed, Jan 1, 2014 at 2:22 PM, Gerrit Jansen van Vuuren <
> > > > > gerrit...@gmail.com> wrote:
> > > > >
> > > > > > The producer is heavily synchronized (i.e. all the code in the
> send
> > > > > method
> > > > > > is encapsulated in one huge synchronized block).
> > > > > > Try creating multiple producers and round robin send over them.
> > > > > >
> > > > > > e.g.
> > > > > >
> > > > > > p = producers[ n++ % producers.length ]
> > > > > >
> > > > > > p.send msg
> > > > > > This will give you one thread per producer instance.
> > > > > >
> > > > > > I'm working on an async multi threaded producer for kafka, but
> its
> > > > > nothing
> > > > > > near complete yet.
> > > > > > https://github.com/gerritjvv/kafka-fast
> > > > > >
> > > > > >
> > > > > > Regards,
> > > > > >  Gerrit
> > > > > >
> > > > > >
> > > > > > On Wed, Jan 1, 2014 at 1:17 PM, yosi botzer <
> yosi.bot...@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Hi,
> > > > > > >
> > > > > > > I am using kafka 0.8. I have 3 machines each running kafka
> > broker.
> > > > > > >
> > > > > > > I am using async mode of my Producer. I expected to see 3
> > different
> > > > > > threads
> > > > > > > with names starting with ProducerSendThread- (according to this
> > > > > article:
> > > > > > > http://engineering.gnip.com/kafka-async-producer/)
> > > > > > >
> > > > > > > However I can see only one thread with the name
> > > *ProducerSendThread-*
> > > > > > >
> > > > > > > This is my producer configuration:
> > > > > > >
> > > > > > > server=1
> > > > > > > topic=dat7
> > > > > > > metadata.broker.list=
> > > > > > > ec2-54-245-111-112.us-west-2.compute.amazonaws.com:9092
> > > > > > > ,ec2-54-245-111-69.us-west-2.compute.amazonaws.com:9092,
> > > > > > > ec2-54-218-183-14.us-west-2.compute.amazonaws.com:9092
> > > > > > > serializer.class=kafka.serializer.DefaultEncoder
> > > > > > > request.required.acks=1
> > > > > > > compression.codec=snappy
> > > > > > > producer.type=async
> > > > > > > queue.buffering.max.ms=2000
> > > > > > > queue.buffering.max.messages=1000
> > > > > > > batch.num.messages=500
> > > > > > >
> > > > > > >
> > > > > > > *What am I missing here?*
> > > > > > >
> > > > > > >
> > > > > > > BTW, I have also experienced very strange behavior regrading my
> > > > > producer
> > > > > > > performance (which may or may not be related to the issue
> above).
> > > > > > >
> > > > > > > When I have defined a topic with 1 partition I got much better
> > > > > throughput
> > > > > > > comparing to a topic with 3 partitions. A producer sending
> > messages
> > > > to
> > > > > a
> > > > > > > topic with 3 partitions had much better throughput comparing
> to a
> > > > topic
> > > > > > > with 12 partitions.
> > > > > > >
> > > > > > > I would expect to have best performance for the topic with 12
> > > > > partitions
> > > > > > > since I have 3 machines running a broker each of with 4 disks
> > (the
> > > > > broker
> > > > > > > is configured to use all 4 disks)
> > > > > > >
> > > > > > > *Is there any logical explanation for this behavior?*
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to