kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jay Kreps <jay.kr...@gmail.com>
Subject Re: benchmark kafka on 10GbE network
Date Thu, 20 Nov 2014 21:05:30 GMT
Great. There is a single I/O thread per producer client that does the
sending so it could be either that the sender thread or that thread is just
pegged. One way to dive in and see what is happening is to add the command
line option
*  -agentlib:hprof=cpu=samples,depth=10*
This will tell us where the time is going. If you sent that around it may
be informative.

-Jay


On Thu, Nov 20, 2014 at 12:41 AM, Manu Zhang <owenzhang1990@gmail.com>
wrote:

> Thanks Jay. The producer metrics from jconsole is quite helpful.
>
> I've switched to the new producer and run producer benchmark with
>
> */usr/lib/kafka/bin/kafka-run-class.sh
> org.apache.kafka.clients.tools.ProducerPerformance topic1 500000000 1000 -1
> acks=1 bootstrap.servers=node1:9092,node2:9092,node3:9092,node4:9092
> buffer.memory=2097152000 batch.size=1000000 linger.ms
> <http://linger.ms>=100*
>
> so my message size is 1000 bytes (gave up on 100 bytes after fruitless
> experiments) and I've deliberately batched outgoing messages with the "
> linger.ms" conf. A single producer could send 300 MB/s on average and 3
> producers almost saturated the network bandwidth. CPU is fully utilized for
> each producer thread. It seems that I can't go further in a single
> producer. Any thoughts ?
>
> Also, I've noticed this kafka-fast
> <https://github.com/gerritjvv/kafka-fast> project,
> who claimed producer throughput could reach 191975 K messages/s for 1KB
> message on 10GbE network. The difference is that a producer is created per
> topic partition.
>
>
> On Wed, Nov 19, 2014 at 12:34 PM, Jay Kreps <jay.kreps@gmail.com> wrote:
>
> > Yeah this will involve some experimentation.
> >
> > The metrics are visible with jconsole or another jmx viewer.
> >
> > It may also be worth looking at the cpu usage per-thread (e.g. start top
> > and press 't' I think).
> >
> > Another simple test for broker vs client as the bottleneck is just to
> start
> > another producer or consumer and see if that improves throughput (if so
> it
> > is probably a client bottleneck).
> >
> > -Jay
> >
> > On Tue, Nov 18, 2014 at 4:44 PM, Manu Zhang <owenzhang1990@gmail.com>
> > wrote:
> >
> > > Thanks Jay for the quick response.
> > >
> > > Yes, it's a single producer and consumer both configured with multiple
> > > threads but I'm not using the new producer.
> > > CPU is typically 50% utilized on client and merely used on broker.
> Disks
> > > aren't busy either as a lot of data are cached in memory.
> > > Would you please give a link for the producer metrics you are referring
> > to
> > > ?
> > >
> > > Thanks,
> > > Manu
> > >
> > > On Wed, Nov 19, 2014 at 2:39 AM, Jay Kreps <jay.kreps@gmail.com>
> wrote:
> > >
> > > > Hey Manu,
> > > >
> > > > I'm not aware of a benchmark on 10GbE. I'd love to see that though.
> > > Diving
> > > > into the results may help us find bottlenecks hidden by the slower
> > > network.
> > > >
> > > > Can you figure out where the bottleneck is in your test? I assume
> this
> > > is a
> > > > single producer and consumer instance and you are using the new
> > producer
> > > as
> > > > in those benchmarks?
> > > >
> > > > This can be slightly tricky as it can be cpu or I/O on either the
> > clients
> > > > or the brokers. You basically have to look at top, iostat, and the
> jmx
> > > > metrics for clues. The producer has good metrics that explain whether
> > it
> > > is
> > > > spending most of its time waiting or sending data. Not sure if there
> > is a
> > > > similar diagnostic for the consumer.
> > > >
> > > > -Jay
> > > >
> > > > On Tue, Nov 18, 2014 at 5:10 AM, Manu Zhang <owenzhang1990@gmail.com
> >
> > > > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I have been trying out kafka benchmarks described in Jay's
> > > > >
> > benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines
> > > > > <
> > > > >
> > > >
> > >
> >
> https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machine
> > > > > >.
> > > > > I'm able to get similar results on a 4-node GbE network whose
> > in-bytes
> > > > > could be saturated at 120MB/s. However, on a 4-node, 10GbE
> network, I
> > > can
> > > > > not get in-bytes higher than 150MB/s. *Has anyone benchmarked kafka
> > on
> > > a
> > > > > 10GbE network ? Any rule of thumb on 10GbE network for
> configurations
> > > of
> > > > > broker, producer and consumer ? *
> > > > >
> > > > > My kafka version is 0.8.1.1 and I've created a topic with 8
> > partitions
> > > > with
> > > > > 1 replica distributed evenly among the 4 nodes. Message size is 100
> > > > bytes.
> > > > > I use all the default kafka settings.
> > > > > My cluster has 4 nodes, where each node has 32 cores, 128MB RAM
> and 3
> > > > disks
> > > > > for kafka.
> > > > >
> > > > > I've tried increasing message size to 1000 bytes which improved
> > > > producer's
> > > > > throughput but not consumer's.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > Manu
> > > > >
> > > >
> > >
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message