Thanks. The producer perf script hasn't changed much. Jun
On Mon, Apr 28, 2014 at 6:55 AM, Bert Corderman <bertc...@gmail.com> wrote: > Sure I will make sure to do a susbset of the tests on 0.8.1. Was anything > changed in the producer test script with this release? > > > > I am in the process of working on some scripts to automate launching > multiple instances of the producer as the throughput is much less then what > the system can support when using a single multi-threaded producer. > > > > Bert > > > On Sun, Apr 27, 2014 at 11:09 PM, Jun Rao <jun...@gmail.com> wrote: > > > Could you run the tests on the 0.8.1.1 release? > > > > Thanks, > > > > Jun > > > > > > On Sat, Apr 26, 2014 at 8:23 PM, Bert Corderman <bertc...@gmail.com> > > wrote: > > > > > version 0.8.0 > > > > > > > > > On Sat, Apr 26, 2014 at 12:03 AM, Jun Rao <jun...@gmail.com> wrote: > > > > > > > Bert, > > > > > > > > Thanks for sharing. Which version of Kafka were you testing? > > > > > > > > > > > > Jun > > > > > > > > > > > > On Fri, Apr 25, 2014 at 3:11 PM, Bert Corderman <bertc...@gmail.com> > > > > wrote: > > > > > > > > > I have been testing kafka for the past week or so and figured I > would > > > > share > > > > > my results so far. > > > > > > > > > > > > > > > I am not sure if the formatting will keep in email but here are the > > > > results > > > > > in a google doc...all 1,100 of them > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://docs.google.com/spreadsheets/d/1UL-o2MiV0gHZtL4jFWNyqRTQl41LFdM0upjRIwCWNgQ/edit?usp=sharing > > > > > > > > > > > > > > > > > > > > One thing I found is there appears to be a bottleneck in > > > > > kafka-producer-perf-test.sh > > > > > > > > > > > > > > > The servers I used for testing have 12 7.2K drives and 16 cores. I > > was > > > > NOT > > > > > unable to scale the broker past 350MBsec when adding drives even > > > though I > > > > > was able to get 150MBsec from a single drive. I wanted to > determine > > > the > > > > > source of the low utilization. > > > > > > > > > > > > > > > I tired changing the following > > > > > > > > > > · log.flush.interval.messages on the broker > > > > > > > > > > · log.flush.interval.ms flush on the broker > > > > > > > > > > · num.io.threads on the broker > > > > > > > > > > · thread settings on the producer > > > > > > > > > > · producer message sizes > > > > > > > > > > · producer batch sizes > > > > > > > > > > · different number of topics (which impact the number of > > drives) > > > > > > > > > > None of the above had any impact. The last thing I tried was > running > > > > > multiple producers which had a very noticeable impact. As > previously > > > > > mentioned I had already tested the thread setting of the producer > and > > > > found > > > > > it to scale when increasing the thread count from 1,2,4 and 8. > After > > > > that > > > > > it plateaued so I had been using 8 threads for each test. To show > > the > > > > > impact on number of producers I created 12 topics with partition > > counts > > > > > from 1 to 12. I used a single broker with no replication and > > > > configured > > > > > the producer(s) to send 10 million 2200 byte messages in batches of > > 400 > > > > > with no ack. > > > > > > > > > > > > > > > Running with three producers has almost double the throughput that > > one > > > > > producer will have. > > > > > > > > > > > > > > > Other Key points learned so far > > > > > > > > > > · Ensure you are using correct network interface. ( use > > > > > advertised.host.name if the servers have multiple interfaces) > > > > > > > > > > · Use batching on the producer – With a single broker > sending > > > 2200 > > > > > byte messages in batches of 200 resulted in 283MBsec vs. a batch > > size > > > > of 1 > > > > > was 44MBsec > > > > > > > > > > · The message size, the configuration of > request.required.acks > > > and > > > > > the number of replicas (only when ack is set to all) had the most > > > > influence > > > > > on the overall throughput. > > > > > > > > > > · The following table shows results of testing with messages > > > sizes > > > > > of 200, 300, 1000 and 2200 bytes on a three node cluster. Each > > message > > > > > size was tested with the three available ack modes (NONE, LEADER > and > > > ALL) > > > > > and with replication of two and three copies. Having three copies > > of > > > > data > > > > > is recommended, however both are included for reference. > > > > > > > > > > *Replica=2* > > > > > > > > > > *Replica=3* > > > > > > > > > > *message.size* > > > > > > > > > > *acks* > > > > > > > > > > *MB.sec* > > > > > > > > > > *nMsg.sec* > > > > > > > > > > *MB.sec* > > > > > > > > > > *nMsg.sec* > > > > > > > > > > *Per Server MB.sec* > > > > > > > > > > *Per Server nMsg.sec* > > > > > > > > > > 200 > > > > > > > > > > NONE > > > > > > > > > > 251 > > > > > > > > > > 1,313,888 > > > > > > > > > > 237 > > > > > > > > > > 1,242,390 > > > > > > > > > > 79 > > > > > > > > > > 414,130 > > > > > > > > > > 300 > > > > > > > > > > NONE > > > > > > > > > > 345 > > > > > > > > > > 1,204,384 > > > > > > > > > > 320 > > > > > > > > > > 1,120,197 > > > > > > > > > > 107 > > > > > > > > > > 373,399 > > > > > > > > > > 1000 > > > > > > > > > > NONE > > > > > > > > > > 522 > > > > > > > > > > 546,896 > > > > > > > > > > 515 > > > > > > > > > > 540,541 > > > > > > > > > > 172 > > > > > > > > > > 180,180 > > > > > > > > > > 2200 > > > > > > > > > > NONE > > > > > > > > > > 368 > > > > > > > > > > 175,165 > > > > > > > > > > 367 > > > > > > > > > > 174,709 > > > > > > > > > > 122 > > > > > > > > > > 58,236 > > > > > > > > > > 200 > > > > > > > > > > LEADER > > > > > > > > > > 115 > > > > > > > > > > 604,376 > > > > > > > > > > 141 > > > > > > > > > > 739,754 > > > > > > > > > > 47 > > > > > > > > > > 246,585 > > > > > > > > > > 300 > > > > > > > > > > LEADER > > > > > > > > > > 186 > > > > > > > > > > 650,280 > > > > > > > > > > 192 > > > > > > > > > > 670,062 > > > > > > > > > > 64 > > > > > > > > > > 223,354 > > > > > > > > > > 1000 > > > > > > > > > > LEADER > > > > > > > > > > 340 > > > > > > > > > > 356,659 > > > > > > > > > > 328 > > > > > > > > > > 343,808 > > > > > > > > > > 109 > > > > > > > > > > 114,603 > > > > > > > > > > 2200 > > > > > > > > > > LEADER > > > > > > > > > > 310 > > > > > > > > > > 147,846 > > > > > > > > > > 293 > > > > > > > > > > 139,729 > > > > > > > > > > 98 > > > > > > > > > > 46,576 > > > > > > > > > > 200 > > > > > > > > > > ALL > > > > > > > > > > 74 > > > > > > > > > > 385,594 > > > > > > > > > > 58 > > > > > > > > > > 304,386 > > > > > > > > > > 19 > > > > > > > > > > 101,462 > > > > > > > > > > 300 > > > > > > > > > > ALL > > > > > > > > > > 105 > > > > > > > > > > 367,282 > > > > > > > > > > 78 > > > > > > > > > > 272,316 > > > > > > > > > > 26 > > > > > > > > > > 90,772 > > > > > > > > > > 1000 > > > > > > > > > > ALL > > > > > > > > > > 203 > > > > > > > > > > 212,400 > > > > > > > > > > 124 > > > > > > > > > > 130,305 > > > > > > > > > > 41 > > > > > > > > > > 43,435 > > > > > > > > > > 2200 > > > > > > > > > > ALL > > > > > > > > > > 212 > > > > > > > > > > 100,820 > > > > > > > > > > 136 > > > > > > > > > > 64,835 > > > > > > > > > > 45 > > > > > > > > > > 21,612 > > > > > > > > > > > > > > > > > > > > Some observations from the above table > > > > > > > > > > · Increasing the number of replicas when > request.required.acks > > > is > > > > > none or leader only has limited impact on overall performance > > > (additional > > > > > resources are required to replicate data but during tests this did > > not > > > > > impact producer throughput) > > > > > > > > > > · Compression is not shown as it was found that the data > > > generated > > > > > for the test is not realistic to a production workload. (GZIP > > > compressed > > > > > data 300:1 which is unrealistic ) > > > > > > > > > > · For some reason a message size of 1000 bytes performed the > > > best. > > > > > Need to look into this more. > > > > > > > > > > > > > > > Thanks > > > > > > > > > > Bert > > > > > > > > > > > > > > >