That should work. I recommend using the performance tool cited in the blog
linked from the "performance" page of the website. That tool is more
accurate and uses the new producer.

On Fri, Apr 24, 2015 at 2:29 PM, Roshan Naik <ros...@hortonworks.com> wrote:

> Can we use the new 0.8.2 producer perf tool against a 0.8.1 broker ?
> -roshan
>
>
> On 4/24/15 1:19 PM, "Jay Kreps" <jay.kr...@gmail.com> wrote:
>
> >Do make sure if you are at all performance sensitive you are using the new
> >producer api we released in 0.8.2.
> >
> >-Jay
> >
> >On Fri, Apr 24, 2015 at 12:46 PM, Roshan Naik <ros...@hortonworks.com>
> >wrote:
> >
> >> Yes, I too notice the same behavior (with producer/consumer perf tool on
> >> 8.1.2) Š adding more threads indeed improved the perf a lot (both with
> >>and
> >> without --sync). in --sync mode
> >>   batch size made almost no diff, larger events improved the perf.
> >>
> >> I was doing some 8.1.2 perf testing with a 1 node broker setup
> >>(machine:
> >> 32 cpu cores, 256gb RAM, 10gig ethernet, 1 x 15000rpm disks,).
> >>
> >> My observations:
> >>
> >>
> >>
> >> ASYNC MODE:
> >>
> >>
> >>
> >>
> >>
> >>
> >>
> >>
> >>
> >>
> >> Partition Count: large improvement when going from 1 to 2, beyond 2 see
> >>a
> >> slight dip
> >>
> >>
> >>
> >>
> >>
> >>
> >>   Number of producer threads: perf much better than sync mode with 1
> >> thread, perf peaks out with ~10 threads, beyond 10 thds perf impacted
> >> negatively
> >>
> >>
> >>
> >> SYNC MODE (does not seem to use batch size)
> >> Batch Size: little to no impact
> >> Event Size: perf scales linearly with event size
> >> Number of producer threads: poor perf with one thread, improves with
> >>more
> >> threads,peaks around 30 to 50 threads
> >> socket.send.buffer.bytes : increasing it Made a small but measurable
> >> difference (~4%)
> >>
> >>
> >> --SYNC mode was much slower.
> >>
> >>
> >> I modified the producer perf tool to use the scala batched producer api
> >> (not available in v8.2) --sync mode and perf of --sync mode was closer
> >>to
> >> async mode.
> >>
> >>
> >> -roshan
> >>
> >>
> >>
> >> On 4/24/15 11:42 AM, "Navneet Gupta (Tech - BLR)"
> >> <navneet.gu...@flipkart.com> wrote:
> >>
> >> >Hi,
> >> >
> >> >I ran some tests on our cluster by sending message from multiple
> >>clients
> >> >(machines). Each machine had about 40-100 threads per producer.
> >> >
> >> >I thought of trying out having multiple producers per clients with each
> >> >producer receiving messages from say 10-15 threads. I actually did see
> >>an
> >> >increase in throughput in this case. It was not one off cases but a
> >> >repeatable phenomenon. I called threads to producer ratio
> >>sharingFactor in
> >> >my code.
> >> >
> >> >I am not planning to use it this way in our clients sending messages to
> >> >Kafka but it did go against the suggestion to have single producer
> >>across
> >> >all threads.
> >> >
> >> >
> >> >
> >> >On Fri, Apr 24, 2015 at 10:27 PM, Manikumar Reddy
> >><ku...@nmsworks.co.in>
> >> >wrote:
> >> >
> >> >> Hi Jay,
> >> >>
> >> >> Yes, we are producing from single process/jvm.
> >> >>
> >> >> From docs "The producer will attempt to batch records together into
> >> >>fewer
> >> >> requests whenever multiple records are being sent to the same
> >> >>partition."
> >> >>
> >> >> If I understand correctly, batching happens at topic/partition level,
> >> >>not
> >> >> at Node level. right?
> >> >>
> >> >> If yes, then  both (single producer for all topics , separate
> >>producer
> >> >>for
> >> >> each topic) approaches
> >> >> may give similar performance.
> >> >>
> >> >> On Fri, Apr 24, 2015 at 9:29 PM, Jay Kreps <jay.kr...@gmail.com>
> >>wrote:
> >> >>
> >> >> > If you are talking about within a single process, having one
> >>producer
> >> >>is
> >> >> > generally the fastest because batching dramatically reduces the
> >> >>number of
> >> >> > requests (esp using the new java producer).
> >> >> > -Jay
> >> >> >
> >> >> > On Fri, Apr 24, 2015 at 4:54 AM, Manikumar Reddy <
> >> >> > manikumar.re...@gmail.com>
> >> >> > wrote:
> >> >> >
> >> >> > > We have a 2 node cluster with 100 topics.
> >> >> > > should we use a single producer for all topics or  create
> >>multiple
> >> >> > > producers?
> >> >> > > What is the best choice w.r.t network load/failures, node
> >>failures,
> >> >> > > latency, locks?
> >> >> > >
> >> >> > > Regards,
> >> >> > > Manikumar
> >> >> > >
> >> >> >
> >> >>
> >> >
> >> >
> >> >
> >> >--
> >> >Thanks & Regards,
> >> >Navneet Gupta
> >>
> >>
>
>

Reply via email to