If you use the perf test without any bound on throughput it will always try to send data faster than it can go out and build up a queue of unsent data. So e.g. if your buffer is 1MB each send will be blocked on waiting for the full 1MB of queued data to clear out and get sent. This makes sense if you think about it.
If you want to test latency under load you need to throttle the maximum throughput to something like what you think you would see in your application (there is an option for that in the command line options). -Jay On Mon, Jan 11, 2016 at 11:02 AM, Andrej Vladimirovich <udodizdu...@gmail.com> wrote: > Ewen, > > One more question. I mentioned that *kafka-run-class.sh > org.apache.kafka.clients.tools.ProducerPerformance* latency is a lot higher > than *kafka-run-class.sh kafka.tools.TestEndToEndLatency.* > > Example: > > *ProducerPerformance:* > > 50000000 records sent, 337463.891364 records/sec (32.18 MB/sec), *1548.51 > ms avg latency*, 3186.00 ms max latency, 2478 ms 50th, 3071 ms 95th, 3118 > ms 99th, 3179 ms 99.9th. > > *TestEndToEndLatency:* > > Percentiles: 50th = 8, 99th = 9, 99.9th = 20 > > So 1548.51 ms vs 9 ms.Huge difference. > > I am using the same cluster, same server and same topic to run both tests. > It does not make any sense to me why would End to End be so low and > Producer to Kafka is so large? > > I did some research online and found other people having the same question > without any responses. > > Thanks a lot for your help! > > Andrew > > On Fri, Jan 8, 2016 at 5:44 PM, Ewen Cheslack-Postava <e...@confluent.io> > wrote: > >> It is single threaded in the sense that you can not request that multiple >> threads be used to call producer.send(). However, the producer has its own >> internal thread for doing network IO. When you have such a simple producer, >> depending on the size of messages you can saturate a 1Gbps link with a >> single thread, so usually using more threads isn't much help. If you still >> need more throughput, you can just use more processes. >> >> -Ewen >> >> On Fri, Jan 8, 2016 at 1:24 PM, Andrej Vladimirovich < >> udodizdu...@gmail.com> >> wrote: >> >> > Thanks Ewen. Do you know if kafka-run-class.sh >> > org.apache.kafka.clients.tools.ProducerPerformance >> > is single threaded? Or is there any way to specify number of threads? >> > >> > On Fri, Jan 8, 2016 at 1:24 PM, Ewen Cheslack-Postava <e...@confluent.io >> > >> > wrote: >> > >> > > Ah, sorry, I missed the version number in your title. I think this tool >> > saw >> > > some rearrangement in 0.9.0 and I was looking at the latest version. >> > > Unfortunately it doesn't look like the old >> > kafka.tools.ProducerPerformance >> > > that is used in kafka-producer-perf-test.sh in 0.8.2.1 supports passing >> > in >> > > additional properties. >> > > >> > > -Ewen >> > > >> > > On Fri, Jan 8, 2016 at 9:10 AM, Andrej Vladimirovich < >> > > udodizdu...@gmail.com> >> > > wrote: >> > > >> > > > Ewen, >> > > > >> > > > I tried that before like this: >> > > > >> > > > ./kafka-producer-perf-test.sh --broker-list test:9092 --topics >> test8-3 >> > > > --messages 2000000 --new-producer --message-size 2000000 >> > > > --show-detailed-stats max.request.size=10000000 >> > > > >> > > > and it does not work. It comletly ignore this option. >> > > > >> > > > And --producer-props is not a valid option for >> > > kafka-producer-perf-test.sh. >> > > > Maybe it is not the right syntax? But I tried a lot of different ways >> > and >> > > > have yet to find the right one. >> > > > >> > > > Thanks! >> > > > >> > > > Andrew >> > > > >> > > > On Fri, Jan 8, 2016 at 10:54 AM, Ewen Cheslack-Postava < >> > > e...@confluent.io> >> > > > wrote: >> > > > >> > > > > Andrew, >> > > > > >> > > > > kafka-producer-perf-test.sh is just a wrapper around >> > > > > orga.apache.kafka.clients.tools.ProducerPerformance and all command >> > > line >> > > > > options should be forwarded. Can you just pass a --producer-props >> to >> > > set >> > > > > max.request.size to a larger value? >> > > > > >> > > > > -Ewen >> > > > > >> > > > > On Fri, Jan 8, 2016 at 7:51 AM, Andrej Vladimirovich < >> > > > > udodizdu...@gmail.com> >> > > > > wrote: >> > > > > >> > > > > > Hi! >> > > > > > >> > > > > > I am testing Kafka's performance with large messages and would >> like >> > > to >> > > > > > specify maximum request size when I run >> > kafka-producer-perf-test.sh: >> > > > > > >> > > > > > ./kafka-producer-perf-test.sh --broker-list "test1:9092" --topics >> > > > test8-3 >> > > > > > --messages 1000000 --new-producer --msage-size 1100000 >> > > > > > --show-detailed-stats >> > > > > > >> > > > > > I always get this message if I specify somethine larger than 1MB: >> > > > > > >> > > > > > ERROR Error when sending message to topic test8-3 with key: 1 >> > bytes, >> > > > > value: >> > > > > > 1100000 bytes with error: The message is 1100027 bytes when >> > > serialized >> > > > > > which is larger than the maximum request size you have configured >> > > with >> > > > > the >> > > > > > max.request.size configuration. >> > > > > > >> (org.apache.kafka.clients.producer.internals.ErrorLoggingCallback) >> > > > > > >> > > > > > I know I can specify maximum request size with kafka-run-class.sh >> > > > > > org.apache.kafka.clients.tools.ProducerPerformance but I would >> like >> > > to >> > > > > use >> > > > > > kafka-producer-perf-test.sh if possible. >> > > > > > >> > > > > > Thanks! >> > > > > > >> > > > > > Andrew >> > > > > > >> > > > > >> > > > > >> > > > > >> > > > > -- >> > > > > Thanks, >> > > > > Ewen >> > > > > >> > > > >> > > >> > > >> > > >> > > -- >> > > Thanks, >> > > Ewen >> > > >> > >> >> >> >> -- >> Thanks, >> Ewen >>