Re: benchmark kafka on 10GbE network

2014-11-27 Thread Manu Zhang
I have 8 partitions / 1 topic. Do you mean hack the new producer to add more threads ? The current implementation manages a single send thread. Thanks, Manu On Wed, Nov 26, 2014 at 1:28 AM, Jay Kreps wrote: > Yeah, neither of those are simple to optimize. The CRC is already the > optimized java

Re: benchmark kafka on 10GbE network

2014-11-25 Thread Jay Kreps
Yeah, neither of those are simple to optimize. The CRC is already the optimized java crc we stole from Hadoop. It may be possible to make that faster still but probably not easy. It might be possible to optimize out some of the interrupt calls, though I'm not exactly sure. One thing, though, is th

Re: benchmark kafka on 10GbE network

2014-11-25 Thread Manu Zhang
Thanks for the explanation. Here are some stats for I/O thread. *io-ratio 0.155* *io-time-ns-avg 16418.5* *io-wait-ratio 0.59* *io-wait-time-ns-avg 62881* It seems to confirm that IO spent much more time waiting than doing real work. Given the above stats, how could I trace down and pinpoint t

Re: benchmark kafka on 10GbE network

2014-11-20 Thread Jay Kreps
So I suspect that the bottleneck is actually in the writer thread (the one calling send()), not the I/O thread. You could verify this by checking the JMX stats which will give the amount of time the I/O thread spends waiting. But since epollWait shows up first that is the I/O thread waiting for wor

Re: benchmark kafka on 10GbE network

2014-11-20 Thread Manu Zhang
Ok, here is the hrpof output CPU SAMPLES BEGIN (total = 202493) Fri Nov 21 08:07:51 2014 rank self accum count trace method 1 39.30% 39.30% 79585 300923 java.net.SocketInputStream.socketRead0 2 20.62% 59.92% 41750 300450 java.net.PlainSocketImpl.socketAccept 3 9.52% 69.45% 192

Re: benchmark kafka on 10GbE network

2014-11-20 Thread Jay Kreps
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 t

Re: benchmark kafka on 10GbE network

2014-11-20 Thread Manu Zhang
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 5 1000 -1 acks=1 bootstrap.servers=node1:9092,node2:9092,node

Re: benchmark kafka on 10GbE network

2014-11-18 Thread Jay Kreps
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

Re: benchmark kafka on 10GbE network

2014-11-18 Thread Manu Zhang
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

Re: benchmark kafka on 10GbE network

2014-11-18 Thread Jay Kreps
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