Re: Reduce latency

2015-08-18 Thread Alvaro Gareppe
you can configure that, to block or to fail:
http://kafka.apache.org/documentation.html#producerconfigs

By default it should block

On Tue, Aug 18, 2015 at 4:57 PM, Yuheng Du  wrote:

> I see. So the internal queue overwrites the producer buffer size
> configuration? When buffer is full the producer will block sending, right?
>
> On Tue, Aug 18, 2015 at 3:52 PM, Tao Feng  wrote:
>
> > From what I understand, if you set the throughput to -1, the
> > producerperformance will push records as much as possible to an internal
> > per topic per partition queue. In the background there is a sender IO
> > thread handling the actual record sending process. If you push record to
> > the queue faster than the send rate, your queue  will become longer and
> > longer, eventually record latency will become meaningless for a
> > latency-purpose test.
> >
> >
> > On Tue, Aug 18, 2015 at 11:48 AM, Yuheng Du 
> > wrote:
> >
> > > I see. Thank you Tao. But now I don't get it what Jay said that my
> > latency
> > > test only makes sense if I set a fixed throughput. Why do I need to
> set a
> > > fixed throughput for my test instead of just set the expected
> throughput
> > to
> > > be -1 (as much as possible)?
> > >
> > > Thanks.
> > >
> > > On Tue, Aug 18, 2015 at 2:43 PM, Tao Feng  wrote:
> > >
> > > > Hi Yuheng,
> > > >
> > > > The 1 record/s is just a param for producerperformance for your
> > > > producer target tput. It only takes effect to do the throttling if
> you
> > > > tries to send more than 1 record/s.  The actual tput of the test
> > > > depends on your producer config and your setup.
> > > >
> > > > -Tao
> > > >
> > > > On Tue, Aug 18, 2015 at 11:34 AM, Yuheng Du <
> yuheng.du.h...@gmail.com>
> > > > wrote:
> > > >
> > > > > Also, When I set the target throughput to be 1 records/s, The
> > > actual
> > > > > test results show I got an average of 579.86 records per second
> among
> > > all
> > > > > my producers. How did that happen? Why this number is not 1
> then?
> > > > > Thanks.
> > > > >
> > > > > On Tue, Aug 18, 2015 at 10:03 AM, Yuheng Du <
> > yuheng.du.h...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Thank you Jay, that really helps!
> > > > > >
> > > > > > Kishore, Where you can monitor whether the network is busy on IO
> in
> > > > > visual
> > > > > > vm? Thanks. I am running 90 producer process on 90 physical
> > machines
> > > in
> > > > > the
> > > > > > experiment.
> > > > > >
> > > > > > On Tue, Aug 18, 2015 at 1:19 AM, Jay Kreps 
> > wrote:
> > > > > >
> > > > > >> Yuheng,
> > > > > >>
> > > > > >> From the command you gave it looks like you are configuring the
> > perf
> > > > > test
> > > > > >> to send data as fast as possible (the -1 for target throughput).
> > > This
> > > > > >> means
> > > > > >> it will always queue up a bunch of unsent data until the buffer
> is
> > > > > >> exhausted and then block. The larger the buffer, the bigger the
> > > queue.
> > > > > >> This
> > > > > >> is where the latency comes from. This is exactly what you would
> > > expect
> > > > > and
> > > > > >> what the buffering is supposed to do.
> > > > > >>
> > > > > >> If you want to measure latency this test doesn't really make
> > sense,
> > > > you
> > > > > >> need to measure with some fixed throughput. Instead of -1 enter
> > the
> > > > > target
> > > > > >> throughput you want to measure latency at (e.g. 10
> > records/sec).
> > > > > >>
> > > > > >> -Jay
> > > > > >>
> > > > > >> On Thu, Aug 13, 2015 at 12:18 PM, Yuheng Du <
> > > yuheng.du.h...@gmail.com
> > > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >> > Thank you Alvaro,
> > > > > >> >
> > > > > >> > How to use sync producers? I am running the standard
> > > > > ProducerPerformance
> > > > > >> > test from kafka to measure the latency of each message to send
> > > from
> > > > > >> > producer to broker only.
> > > > > >> > The command is like "bin/kafka-run-class.sh
> > > > > >> > org.apache.kafka.clients.tools.ProducerPerformance test7
> > 5000
> > > > 100
> > > > > -1
> > > > > >> > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > > > > >> > buffer.memory=67108864 batch.size=8196"
> > > > > >> >
> > > > > >> > For running producers, where should I put the
> producer.type=sync
> > > > > >> > configuration into? The config/server.properties? Also Does
> this
> > > > mean
> > > > > we
> > > > > >> > are using batch size of 1? Which version of Kafka are you
> using?
> > > > > >> > thanks.
> > > > > >> >
> > > > > >> > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe <
> > > agare...@gmail.com
> > > > >
> > > > > >> > wrote:
> > > > > >> >
> > > > > >> > > Are you measuring latency as time between producer and
> > consumer
> > > ?
> > > > > >> > >
> > > > > >> > > In that case, the ack shouldn't affect the latency, cause
> even
> > > > tough
> > > > > >> your
> > > > > >> > > producer is not going to wait for the ack, the consumer will
> > > only
> > > > > get
> > > > > >> the
> > > > > >> > > messag

Re: Reduce latency

2015-08-18 Thread Yuheng Du
I see. So the internal queue overwrites the producer buffer size
configuration? When buffer is full the producer will block sending, right?

On Tue, Aug 18, 2015 at 3:52 PM, Tao Feng  wrote:

> From what I understand, if you set the throughput to -1, the
> producerperformance will push records as much as possible to an internal
> per topic per partition queue. In the background there is a sender IO
> thread handling the actual record sending process. If you push record to
> the queue faster than the send rate, your queue  will become longer and
> longer, eventually record latency will become meaningless for a
> latency-purpose test.
>
>
> On Tue, Aug 18, 2015 at 11:48 AM, Yuheng Du 
> wrote:
>
> > I see. Thank you Tao. But now I don't get it what Jay said that my
> latency
> > test only makes sense if I set a fixed throughput. Why do I need to set a
> > fixed throughput for my test instead of just set the expected throughput
> to
> > be -1 (as much as possible)?
> >
> > Thanks.
> >
> > On Tue, Aug 18, 2015 at 2:43 PM, Tao Feng  wrote:
> >
> > > Hi Yuheng,
> > >
> > > The 1 record/s is just a param for producerperformance for your
> > > producer target tput. It only takes effect to do the throttling if you
> > > tries to send more than 1 record/s.  The actual tput of the test
> > > depends on your producer config and your setup.
> > >
> > > -Tao
> > >
> > > On Tue, Aug 18, 2015 at 11:34 AM, Yuheng Du 
> > > wrote:
> > >
> > > > Also, When I set the target throughput to be 1 records/s, The
> > actual
> > > > test results show I got an average of 579.86 records per second among
> > all
> > > > my producers. How did that happen? Why this number is not 1 then?
> > > > Thanks.
> > > >
> > > > On Tue, Aug 18, 2015 at 10:03 AM, Yuheng Du <
> yuheng.du.h...@gmail.com>
> > > > wrote:
> > > >
> > > > > Thank you Jay, that really helps!
> > > > >
> > > > > Kishore, Where you can monitor whether the network is busy on IO in
> > > > visual
> > > > > vm? Thanks. I am running 90 producer process on 90 physical
> machines
> > in
> > > > the
> > > > > experiment.
> > > > >
> > > > > On Tue, Aug 18, 2015 at 1:19 AM, Jay Kreps 
> wrote:
> > > > >
> > > > >> Yuheng,
> > > > >>
> > > > >> From the command you gave it looks like you are configuring the
> perf
> > > > test
> > > > >> to send data as fast as possible (the -1 for target throughput).
> > This
> > > > >> means
> > > > >> it will always queue up a bunch of unsent data until the buffer is
> > > > >> exhausted and then block. The larger the buffer, the bigger the
> > queue.
> > > > >> This
> > > > >> is where the latency comes from. This is exactly what you would
> > expect
> > > > and
> > > > >> what the buffering is supposed to do.
> > > > >>
> > > > >> If you want to measure latency this test doesn't really make
> sense,
> > > you
> > > > >> need to measure with some fixed throughput. Instead of -1 enter
> the
> > > > target
> > > > >> throughput you want to measure latency at (e.g. 10
> records/sec).
> > > > >>
> > > > >> -Jay
> > > > >>
> > > > >> On Thu, Aug 13, 2015 at 12:18 PM, Yuheng Du <
> > yuheng.du.h...@gmail.com
> > > >
> > > > >> wrote:
> > > > >>
> > > > >> > Thank you Alvaro,
> > > > >> >
> > > > >> > How to use sync producers? I am running the standard
> > > > ProducerPerformance
> > > > >> > test from kafka to measure the latency of each message to send
> > from
> > > > >> > producer to broker only.
> > > > >> > The command is like "bin/kafka-run-class.sh
> > > > >> > org.apache.kafka.clients.tools.ProducerPerformance test7
> 5000
> > > 100
> > > > -1
> > > > >> > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > > > >> > buffer.memory=67108864 batch.size=8196"
> > > > >> >
> > > > >> > For running producers, where should I put the producer.type=sync
> > > > >> > configuration into? The config/server.properties? Also Does this
> > > mean
> > > > we
> > > > >> > are using batch size of 1? Which version of Kafka are you using?
> > > > >> > thanks.
> > > > >> >
> > > > >> > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe <
> > agare...@gmail.com
> > > >
> > > > >> > wrote:
> > > > >> >
> > > > >> > > Are you measuring latency as time between producer and
> consumer
> > ?
> > > > >> > >
> > > > >> > > In that case, the ack shouldn't affect the latency, cause even
> > > tough
> > > > >> your
> > > > >> > > producer is not going to wait for the ack, the consumer will
> > only
> > > > get
> > > > >> the
> > > > >> > > message after its commited in the server.
> > > > >> > >
> > > > >> > > About latency my best result occur with sync producers, but
> the
> > > > >> > throughput
> > > > >> > > is much lower in that case.
> > > > >> > >
> > > > >> > > About not flushing to disk I'm pretty sure that it's not an
> > option
> > > > in
> > > > >> > kafka
> > > > >> > > (correct me if I'm wrong)
> > > > >> > >
> > > > >> > > Regards,
> > > > >> > > Alvaro Gareppe
> > > > >> > >
> > > > >> > > On Thu, Aug 13, 2015 at 12:59 PM, Yu

Re: Reduce latency

2015-08-18 Thread Tao Feng
>From what I understand, if you set the throughput to -1, the
producerperformance will push records as much as possible to an internal
per topic per partition queue. In the background there is a sender IO
thread handling the actual record sending process. If you push record to
the queue faster than the send rate, your queue  will become longer and
longer, eventually record latency will become meaningless for a
latency-purpose test.


On Tue, Aug 18, 2015 at 11:48 AM, Yuheng Du 
wrote:

> I see. Thank you Tao. But now I don't get it what Jay said that my latency
> test only makes sense if I set a fixed throughput. Why do I need to set a
> fixed throughput for my test instead of just set the expected throughput to
> be -1 (as much as possible)?
>
> Thanks.
>
> On Tue, Aug 18, 2015 at 2:43 PM, Tao Feng  wrote:
>
> > Hi Yuheng,
> >
> > The 1 record/s is just a param for producerperformance for your
> > producer target tput. It only takes effect to do the throttling if you
> > tries to send more than 1 record/s.  The actual tput of the test
> > depends on your producer config and your setup.
> >
> > -Tao
> >
> > On Tue, Aug 18, 2015 at 11:34 AM, Yuheng Du 
> > wrote:
> >
> > > Also, When I set the target throughput to be 1 records/s, The
> actual
> > > test results show I got an average of 579.86 records per second among
> all
> > > my producers. How did that happen? Why this number is not 1 then?
> > > Thanks.
> > >
> > > On Tue, Aug 18, 2015 at 10:03 AM, Yuheng Du 
> > > wrote:
> > >
> > > > Thank you Jay, that really helps!
> > > >
> > > > Kishore, Where you can monitor whether the network is busy on IO in
> > > visual
> > > > vm? Thanks. I am running 90 producer process on 90 physical machines
> in
> > > the
> > > > experiment.
> > > >
> > > > On Tue, Aug 18, 2015 at 1:19 AM, Jay Kreps  wrote:
> > > >
> > > >> Yuheng,
> > > >>
> > > >> From the command you gave it looks like you are configuring the perf
> > > test
> > > >> to send data as fast as possible (the -1 for target throughput).
> This
> > > >> means
> > > >> it will always queue up a bunch of unsent data until the buffer is
> > > >> exhausted and then block. The larger the buffer, the bigger the
> queue.
> > > >> This
> > > >> is where the latency comes from. This is exactly what you would
> expect
> > > and
> > > >> what the buffering is supposed to do.
> > > >>
> > > >> If you want to measure latency this test doesn't really make sense,
> > you
> > > >> need to measure with some fixed throughput. Instead of -1 enter the
> > > target
> > > >> throughput you want to measure latency at (e.g. 10 records/sec).
> > > >>
> > > >> -Jay
> > > >>
> > > >> On Thu, Aug 13, 2015 at 12:18 PM, Yuheng Du <
> yuheng.du.h...@gmail.com
> > >
> > > >> wrote:
> > > >>
> > > >> > Thank you Alvaro,
> > > >> >
> > > >> > How to use sync producers? I am running the standard
> > > ProducerPerformance
> > > >> > test from kafka to measure the latency of each message to send
> from
> > > >> > producer to broker only.
> > > >> > The command is like "bin/kafka-run-class.sh
> > > >> > org.apache.kafka.clients.tools.ProducerPerformance test7 5000
> > 100
> > > -1
> > > >> > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > > >> > buffer.memory=67108864 batch.size=8196"
> > > >> >
> > > >> > For running producers, where should I put the producer.type=sync
> > > >> > configuration into? The config/server.properties? Also Does this
> > mean
> > > we
> > > >> > are using batch size of 1? Which version of Kafka are you using?
> > > >> > thanks.
> > > >> >
> > > >> > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe <
> agare...@gmail.com
> > >
> > > >> > wrote:
> > > >> >
> > > >> > > Are you measuring latency as time between producer and consumer
> ?
> > > >> > >
> > > >> > > In that case, the ack shouldn't affect the latency, cause even
> > tough
> > > >> your
> > > >> > > producer is not going to wait for the ack, the consumer will
> only
> > > get
> > > >> the
> > > >> > > message after its commited in the server.
> > > >> > >
> > > >> > > About latency my best result occur with sync producers, but the
> > > >> > throughput
> > > >> > > is much lower in that case.
> > > >> > >
> > > >> > > About not flushing to disk I'm pretty sure that it's not an
> option
> > > in
> > > >> > kafka
> > > >> > > (correct me if I'm wrong)
> > > >> > >
> > > >> > > Regards,
> > > >> > > Alvaro Gareppe
> > > >> > >
> > > >> > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du <
> > > yuheng.du.h...@gmail.com
> > > >> >
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Also, the latency results show no major difference when using
> > > ack=0
> > > >> or
> > > >> > > > ack=1. Why is that?
> > > >> > > >
> > > >> > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
> > > >> yuheng.du.h...@gmail.com>
> > > >> > > > wrote:
> > > >> > > >
> > > >> > > > > I am running an experiment where 92 producers is publishing
> > data
> > > >> > into 6
> > > >> > > > > brokers and 10 cons

Re: Reduce latency

2015-08-18 Thread Yuheng Du
I see. Thank you Tao. But now I don't get it what Jay said that my latency
test only makes sense if I set a fixed throughput. Why do I need to set a
fixed throughput for my test instead of just set the expected throughput to
be -1 (as much as possible)?

Thanks.

On Tue, Aug 18, 2015 at 2:43 PM, Tao Feng  wrote:

> Hi Yuheng,
>
> The 1 record/s is just a param for producerperformance for your
> producer target tput. It only takes effect to do the throttling if you
> tries to send more than 1 record/s.  The actual tput of the test
> depends on your producer config and your setup.
>
> -Tao
>
> On Tue, Aug 18, 2015 at 11:34 AM, Yuheng Du 
> wrote:
>
> > Also, When I set the target throughput to be 1 records/s, The actual
> > test results show I got an average of 579.86 records per second among all
> > my producers. How did that happen? Why this number is not 1 then?
> > Thanks.
> >
> > On Tue, Aug 18, 2015 at 10:03 AM, Yuheng Du 
> > wrote:
> >
> > > Thank you Jay, that really helps!
> > >
> > > Kishore, Where you can monitor whether the network is busy on IO in
> > visual
> > > vm? Thanks. I am running 90 producer process on 90 physical machines in
> > the
> > > experiment.
> > >
> > > On Tue, Aug 18, 2015 at 1:19 AM, Jay Kreps  wrote:
> > >
> > >> Yuheng,
> > >>
> > >> From the command you gave it looks like you are configuring the perf
> > test
> > >> to send data as fast as possible (the -1 for target throughput). This
> > >> means
> > >> it will always queue up a bunch of unsent data until the buffer is
> > >> exhausted and then block. The larger the buffer, the bigger the queue.
> > >> This
> > >> is where the latency comes from. This is exactly what you would expect
> > and
> > >> what the buffering is supposed to do.
> > >>
> > >> If you want to measure latency this test doesn't really make sense,
> you
> > >> need to measure with some fixed throughput. Instead of -1 enter the
> > target
> > >> throughput you want to measure latency at (e.g. 10 records/sec).
> > >>
> > >> -Jay
> > >>
> > >> On Thu, Aug 13, 2015 at 12:18 PM, Yuheng Du  >
> > >> wrote:
> > >>
> > >> > Thank you Alvaro,
> > >> >
> > >> > How to use sync producers? I am running the standard
> > ProducerPerformance
> > >> > test from kafka to measure the latency of each message to send from
> > >> > producer to broker only.
> > >> > The command is like "bin/kafka-run-class.sh
> > >> > org.apache.kafka.clients.tools.ProducerPerformance test7 5000
> 100
> > -1
> > >> > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > >> > buffer.memory=67108864 batch.size=8196"
> > >> >
> > >> > For running producers, where should I put the producer.type=sync
> > >> > configuration into? The config/server.properties? Also Does this
> mean
> > we
> > >> > are using batch size of 1? Which version of Kafka are you using?
> > >> > thanks.
> > >> >
> > >> > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe  >
> > >> > wrote:
> > >> >
> > >> > > Are you measuring latency as time between producer and consumer ?
> > >> > >
> > >> > > In that case, the ack shouldn't affect the latency, cause even
> tough
> > >> your
> > >> > > producer is not going to wait for the ack, the consumer will only
> > get
> > >> the
> > >> > > message after its commited in the server.
> > >> > >
> > >> > > About latency my best result occur with sync producers, but the
> > >> > throughput
> > >> > > is much lower in that case.
> > >> > >
> > >> > > About not flushing to disk I'm pretty sure that it's not an option
> > in
> > >> > kafka
> > >> > > (correct me if I'm wrong)
> > >> > >
> > >> > > Regards,
> > >> > > Alvaro Gareppe
> > >> > >
> > >> > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du <
> > yuheng.du.h...@gmail.com
> > >> >
> > >> > > wrote:
> > >> > >
> > >> > > > Also, the latency results show no major difference when using
> > ack=0
> > >> or
> > >> > > > ack=1. Why is that?
> > >> > > >
> > >> > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
> > >> yuheng.du.h...@gmail.com>
> > >> > > > wrote:
> > >> > > >
> > >> > > > > I am running an experiment where 92 producers is publishing
> data
> > >> > into 6
> > >> > > > > brokers and 10 consumer are reading online data
> simultaneously.
> > >> > > > >
> > >> > > > > How should I do to reduce the latency? Currently when I run
> the
> > >> > > producer
> > >> > > > > performance test the average latency is around 10s.
> > >> > > > >
> > >> > > > > Should I disable log.flush? How to do that? Thanks.
> > >> > > > >
> > >> > > >
> > >> > >
> > >> > >
> > >> > >
> > >> > > --
> > >> > > Ing. Alvaro Gareppe
> > >> > > agare...@gmail.com
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>


Re: Reduce latency

2015-08-18 Thread Tao Feng
Hi Yuheng,

The 1 record/s is just a param for producerperformance for your
producer target tput. It only takes effect to do the throttling if you
tries to send more than 1 record/s.  The actual tput of the test
depends on your producer config and your setup.

-Tao

On Tue, Aug 18, 2015 at 11:34 AM, Yuheng Du 
wrote:

> Also, When I set the target throughput to be 1 records/s, The actual
> test results show I got an average of 579.86 records per second among all
> my producers. How did that happen? Why this number is not 1 then?
> Thanks.
>
> On Tue, Aug 18, 2015 at 10:03 AM, Yuheng Du 
> wrote:
>
> > Thank you Jay, that really helps!
> >
> > Kishore, Where you can monitor whether the network is busy on IO in
> visual
> > vm? Thanks. I am running 90 producer process on 90 physical machines in
> the
> > experiment.
> >
> > On Tue, Aug 18, 2015 at 1:19 AM, Jay Kreps  wrote:
> >
> >> Yuheng,
> >>
> >> From the command you gave it looks like you are configuring the perf
> test
> >> to send data as fast as possible (the -1 for target throughput). This
> >> means
> >> it will always queue up a bunch of unsent data until the buffer is
> >> exhausted and then block. The larger the buffer, the bigger the queue.
> >> This
> >> is where the latency comes from. This is exactly what you would expect
> and
> >> what the buffering is supposed to do.
> >>
> >> If you want to measure latency this test doesn't really make sense, you
> >> need to measure with some fixed throughput. Instead of -1 enter the
> target
> >> throughput you want to measure latency at (e.g. 10 records/sec).
> >>
> >> -Jay
> >>
> >> On Thu, Aug 13, 2015 at 12:18 PM, Yuheng Du 
> >> wrote:
> >>
> >> > Thank you Alvaro,
> >> >
> >> > How to use sync producers? I am running the standard
> ProducerPerformance
> >> > test from kafka to measure the latency of each message to send from
> >> > producer to broker only.
> >> > The command is like "bin/kafka-run-class.sh
> >> > org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100
> -1
> >> > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> >> > buffer.memory=67108864 batch.size=8196"
> >> >
> >> > For running producers, where should I put the producer.type=sync
> >> > configuration into? The config/server.properties? Also Does this mean
> we
> >> > are using batch size of 1? Which version of Kafka are you using?
> >> > thanks.
> >> >
> >> > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe 
> >> > wrote:
> >> >
> >> > > Are you measuring latency as time between producer and consumer ?
> >> > >
> >> > > In that case, the ack shouldn't affect the latency, cause even tough
> >> your
> >> > > producer is not going to wait for the ack, the consumer will only
> get
> >> the
> >> > > message after its commited in the server.
> >> > >
> >> > > About latency my best result occur with sync producers, but the
> >> > throughput
> >> > > is much lower in that case.
> >> > >
> >> > > About not flushing to disk I'm pretty sure that it's not an option
> in
> >> > kafka
> >> > > (correct me if I'm wrong)
> >> > >
> >> > > Regards,
> >> > > Alvaro Gareppe
> >> > >
> >> > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du <
> yuheng.du.h...@gmail.com
> >> >
> >> > > wrote:
> >> > >
> >> > > > Also, the latency results show no major difference when using
> ack=0
> >> or
> >> > > > ack=1. Why is that?
> >> > > >
> >> > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
> >> yuheng.du.h...@gmail.com>
> >> > > > wrote:
> >> > > >
> >> > > > > I am running an experiment where 92 producers is publishing data
> >> > into 6
> >> > > > > brokers and 10 consumer are reading online data simultaneously.
> >> > > > >
> >> > > > > How should I do to reduce the latency? Currently when I run the
> >> > > producer
> >> > > > > performance test the average latency is around 10s.
> >> > > > >
> >> > > > > Should I disable log.flush? How to do that? Thanks.
> >> > > > >
> >> > > >
> >> > >
> >> > >
> >> > >
> >> > > --
> >> > > Ing. Alvaro Gareppe
> >> > > agare...@gmail.com
> >> > >
> >> >
> >>
> >
> >
>


Re: Reduce latency

2015-08-18 Thread Yuheng Du
Also, When I set the target throughput to be 1 records/s, The actual
test results show I got an average of 579.86 records per second among all
my producers. How did that happen? Why this number is not 1 then?
Thanks.

On Tue, Aug 18, 2015 at 10:03 AM, Yuheng Du 
wrote:

> Thank you Jay, that really helps!
>
> Kishore, Where you can monitor whether the network is busy on IO in visual
> vm? Thanks. I am running 90 producer process on 90 physical machines in the
> experiment.
>
> On Tue, Aug 18, 2015 at 1:19 AM, Jay Kreps  wrote:
>
>> Yuheng,
>>
>> From the command you gave it looks like you are configuring the perf test
>> to send data as fast as possible (the -1 for target throughput). This
>> means
>> it will always queue up a bunch of unsent data until the buffer is
>> exhausted and then block. The larger the buffer, the bigger the queue.
>> This
>> is where the latency comes from. This is exactly what you would expect and
>> what the buffering is supposed to do.
>>
>> If you want to measure latency this test doesn't really make sense, you
>> need to measure with some fixed throughput. Instead of -1 enter the target
>> throughput you want to measure latency at (e.g. 10 records/sec).
>>
>> -Jay
>>
>> On Thu, Aug 13, 2015 at 12:18 PM, Yuheng Du 
>> wrote:
>>
>> > Thank you Alvaro,
>> >
>> > How to use sync producers? I am running the standard ProducerPerformance
>> > test from kafka to measure the latency of each message to send from
>> > producer to broker only.
>> > The command is like "bin/kafka-run-class.sh
>> > org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100 -1
>> > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
>> > buffer.memory=67108864 batch.size=8196"
>> >
>> > For running producers, where should I put the producer.type=sync
>> > configuration into? The config/server.properties? Also Does this mean we
>> > are using batch size of 1? Which version of Kafka are you using?
>> > thanks.
>> >
>> > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe 
>> > wrote:
>> >
>> > > Are you measuring latency as time between producer and consumer ?
>> > >
>> > > In that case, the ack shouldn't affect the latency, cause even tough
>> your
>> > > producer is not going to wait for the ack, the consumer will only get
>> the
>> > > message after its commited in the server.
>> > >
>> > > About latency my best result occur with sync producers, but the
>> > throughput
>> > > is much lower in that case.
>> > >
>> > > About not flushing to disk I'm pretty sure that it's not an option in
>> > kafka
>> > > (correct me if I'm wrong)
>> > >
>> > > Regards,
>> > > Alvaro Gareppe
>> > >
>> > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du > >
>> > > wrote:
>> > >
>> > > > Also, the latency results show no major difference when using ack=0
>> or
>> > > > ack=1. Why is that?
>> > > >
>> > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
>> yuheng.du.h...@gmail.com>
>> > > > wrote:
>> > > >
>> > > > > I am running an experiment where 92 producers is publishing data
>> > into 6
>> > > > > brokers and 10 consumer are reading online data simultaneously.
>> > > > >
>> > > > > How should I do to reduce the latency? Currently when I run the
>> > > producer
>> > > > > performance test the average latency is around 10s.
>> > > > >
>> > > > > Should I disable log.flush? How to do that? Thanks.
>> > > > >
>> > > >
>> > >
>> > >
>> > >
>> > > --
>> > > Ing. Alvaro Gareppe
>> > > agare...@gmail.com
>> > >
>> >
>>
>
>


Re: Reduce latency

2015-08-18 Thread Yuheng Du
Thank you Jay, that really helps!

Kishore, Where you can monitor whether the network is busy on IO in visual
vm? Thanks. I am running 90 producer process on 90 physical machines in the
experiment.

On Tue, Aug 18, 2015 at 1:19 AM, Jay Kreps  wrote:

> Yuheng,
>
> From the command you gave it looks like you are configuring the perf test
> to send data as fast as possible (the -1 for target throughput). This means
> it will always queue up a bunch of unsent data until the buffer is
> exhausted and then block. The larger the buffer, the bigger the queue. This
> is where the latency comes from. This is exactly what you would expect and
> what the buffering is supposed to do.
>
> If you want to measure latency this test doesn't really make sense, you
> need to measure with some fixed throughput. Instead of -1 enter the target
> throughput you want to measure latency at (e.g. 10 records/sec).
>
> -Jay
>
> On Thu, Aug 13, 2015 at 12:18 PM, Yuheng Du 
> wrote:
>
> > Thank you Alvaro,
> >
> > How to use sync producers? I am running the standard ProducerPerformance
> > test from kafka to measure the latency of each message to send from
> > producer to broker only.
> > The command is like "bin/kafka-run-class.sh
> > org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100 -1
> > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > buffer.memory=67108864 batch.size=8196"
> >
> > For running producers, where should I put the producer.type=sync
> > configuration into? The config/server.properties? Also Does this mean we
> > are using batch size of 1? Which version of Kafka are you using?
> > thanks.
> >
> > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe 
> > wrote:
> >
> > > Are you measuring latency as time between producer and consumer ?
> > >
> > > In that case, the ack shouldn't affect the latency, cause even tough
> your
> > > producer is not going to wait for the ack, the consumer will only get
> the
> > > message after its commited in the server.
> > >
> > > About latency my best result occur with sync producers, but the
> > throughput
> > > is much lower in that case.
> > >
> > > About not flushing to disk I'm pretty sure that it's not an option in
> > kafka
> > > (correct me if I'm wrong)
> > >
> > > Regards,
> > > Alvaro Gareppe
> > >
> > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du 
> > > wrote:
> > >
> > > > Also, the latency results show no major difference when using ack=0
> or
> > > > ack=1. Why is that?
> > > >
> > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
> yuheng.du.h...@gmail.com>
> > > > wrote:
> > > >
> > > > > I am running an experiment where 92 producers is publishing data
> > into 6
> > > > > brokers and 10 consumer are reading online data simultaneously.
> > > > >
> > > > > How should I do to reduce the latency? Currently when I run the
> > > producer
> > > > > performance test the average latency is around 10s.
> > > > >
> > > > > Should I disable log.flush? How to do that? Thanks.
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Ing. Alvaro Gareppe
> > > agare...@gmail.com
> > >
> >
>


Re: Reduce latency

2015-08-17 Thread Jay Kreps
Yuheng,

>From the command you gave it looks like you are configuring the perf test
to send data as fast as possible (the -1 for target throughput). This means
it will always queue up a bunch of unsent data until the buffer is
exhausted and then block. The larger the buffer, the bigger the queue. This
is where the latency comes from. This is exactly what you would expect and
what the buffering is supposed to do.

If you want to measure latency this test doesn't really make sense, you
need to measure with some fixed throughput. Instead of -1 enter the target
throughput you want to measure latency at (e.g. 10 records/sec).

-Jay

On Thu, Aug 13, 2015 at 12:18 PM, Yuheng Du 
wrote:

> Thank you Alvaro,
>
> How to use sync producers? I am running the standard ProducerPerformance
> test from kafka to measure the latency of each message to send from
> producer to broker only.
> The command is like "bin/kafka-run-class.sh
> org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100 -1
> acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> buffer.memory=67108864 batch.size=8196"
>
> For running producers, where should I put the producer.type=sync
> configuration into? The config/server.properties? Also Does this mean we
> are using batch size of 1? Which version of Kafka are you using?
> thanks.
>
> On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe 
> wrote:
>
> > Are you measuring latency as time between producer and consumer ?
> >
> > In that case, the ack shouldn't affect the latency, cause even tough your
> > producer is not going to wait for the ack, the consumer will only get the
> > message after its commited in the server.
> >
> > About latency my best result occur with sync producers, but the
> throughput
> > is much lower in that case.
> >
> > About not flushing to disk I'm pretty sure that it's not an option in
> kafka
> > (correct me if I'm wrong)
> >
> > Regards,
> > Alvaro Gareppe
> >
> > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du 
> > wrote:
> >
> > > Also, the latency results show no major difference when using ack=0 or
> > > ack=1. Why is that?
> > >
> > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du 
> > > wrote:
> > >
> > > > I am running an experiment where 92 producers is publishing data
> into 6
> > > > brokers and 10 consumer are reading online data simultaneously.
> > > >
> > > > How should I do to reduce the latency? Currently when I run the
> > producer
> > > > performance test the average latency is around 10s.
> > > >
> > > > Should I disable log.flush? How to do that? Thanks.
> > > >
> > >
> >
> >
> >
> > --
> > Ing. Alvaro Gareppe
> > agare...@gmail.com
> >
>


Re: Reduce latency

2015-08-17 Thread Kishore Senji
Just to clarify it is 1 thread / Broker / Producer. Javadoc recommends to
use one Producer across multiple threads and batching is done behind the
scenes so throughput should not be a problem. So if you have more brokers,
the same Producer instance will use multiple threads to send to each
Broker, this way load is also spread out on the brokers and you will get a
good performance from the Producer.

But if you notice that the network thread is busy on IO all the time (in
visual vm) and there is more bandwidth that you can leverage and the
Brokers can handle more requests, then you can go for multiple instances of
Producers (may be a pool of Producers). But this will also increase the
memory usage as the buffer for batching is per Producer.

On Mon, Aug 17, 2015 at 9:55 PM Tao Feng  wrote:

> If you run producerPerformance test, there is only one thread per
> KafkaProducer doing the actual sending. But one network request could
> contain multiple batches from what I understand.
>
> On Mon, Aug 17, 2015 at 5:42 PM, Yuheng Du 
> wrote:
>
> > Thank you Kishore, I made the buffer twice the size of the batch size and
> > the latency has reduced significantly.
> >
> > But is there only one thread io thread sending the batches? Can I
> increase
> > the number of threads sending the batches so more than one batch could be
> > sent at the same time?
> >
> > Thanks.
> >
> >
> >
> > On Thu, Aug 13, 2015 at 5:38 PM, Kishore Senji  wrote:
> >
> > > Your batch.size is 8196 and your buffer.memory is 67108864. This means
> > > 67108864/8196
> > > ~ 8188 batches are in memory ready to the sent. There is only one
> thread
> > io
> > > thread sending them. I would guess that the io thread (
> > > kafka-producer-network-thread) would be busy. Please check it in visual
> > vm.
> > >
> > > In steady state, every batch has to wait for the previous 8187 batches
> to
> > > be done before it gets a chance to be sent out, but the latency is
> > counted
> > > from the time is added to the queue. This is the reason that you are
> > seeing
> > > very high end-to-end latency.
> > >
> > > Have the buffer.memory to be only twice that of the batch.size so that
> > > while one is in flight, you can another batch ready to go (and the
> > > KafkaProducer would block to send more when there is no memory and this
> > way
> > > the batches are not waiting in the queue unnecessarily) . Also may be
> you
> > > want to increase the batch.size further more, you will get even better
> > > throughput with more or less same latency (as there is no shortage of
> > > events in the test program).
> > >
> > > On Thu, Aug 13, 2015 at 1:13 PM Yuheng Du 
> > > wrote:
> > >
> > > > Yes there is. But if we are using ProducerPerformance test, it's
> > > configured
> > > > as giving input when running the test command. Do you write a java
> > > program
> > > > to test the latency? Thanks.
> > > >
> > > > On Thu, Aug 13, 2015 at 3:54 PM, Alvaro Gareppe 
> > > > wrote:
> > > >
> > > > > I'm using last one, but not using the ProducerPerformance, I
> created
> > my
> > > > > own. but I think there is a producer.properties file in config
> folder
> > > in
> > > > > kafka.. is that configuration not for this tester ?
> > > > >
> > > > > On Thu, Aug 13, 2015 at 4:18 PM, Yuheng Du <
> yuheng.du.h...@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > Thank you Alvaro,
> > > > > >
> > > > > > How to use sync producers? I am running the standard
> > > > ProducerPerformance
> > > > > > test from kafka to measure the latency of each message to send
> from
> > > > > > producer to broker only.
> > > > > > The command is like "bin/kafka-run-class.sh
> > > > > > org.apache.kafka.clients.tools.ProducerPerformance test7 5000
> > 100
> > > > -1
> > > > > > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > > > > > buffer.memory=67108864 batch.size=8196"
> > > > > >
> > > > > > For running producers, where should I put the producer.type=sync
> > > > > > configuration into? The config/server.properties? Also Does this
> > mean
> > > > we
> > > > > > are using batch size of 1? Which version of Kafka are you using?
> > > > > > thanks.
> > > > > >
> > > > > > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe <
> > agare...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Are you measuring latency as time between producer and
> consumer ?
> > > > > > >
> > > > > > > In that case, the ack shouldn't affect the latency, cause even
> > > tough
> > > > > your
> > > > > > > producer is not going to wait for the ack, the consumer will
> only
> > > get
> > > > > the
> > > > > > > message after its commited in the server.
> > > > > > >
> > > > > > > About latency my best result occur with sync producers, but the
> > > > > > throughput
> > > > > > > is much lower in that case.
> > > > > > >
> > > > > > > About not flushing to disk I'm pretty sure that it's not an
> > option
> > > in
> > > > > > kafka
> > > > > > > (correct me if I'm wrong)
> > > > > > >
> > > > > > > Regard

Re: Reduce latency

2015-08-17 Thread Tao Feng
If you run producerPerformance test, there is only one thread per
KafkaProducer doing the actual sending. But one network request could
contain multiple batches from what I understand.

On Mon, Aug 17, 2015 at 5:42 PM, Yuheng Du  wrote:

> Thank you Kishore, I made the buffer twice the size of the batch size and
> the latency has reduced significantly.
>
> But is there only one thread io thread sending the batches? Can I increase
> the number of threads sending the batches so more than one batch could be
> sent at the same time?
>
> Thanks.
>
>
>
> On Thu, Aug 13, 2015 at 5:38 PM, Kishore Senji  wrote:
>
> > Your batch.size is 8196 and your buffer.memory is 67108864. This means
> > 67108864/8196
> > ~ 8188 batches are in memory ready to the sent. There is only one thread
> io
> > thread sending them. I would guess that the io thread (
> > kafka-producer-network-thread) would be busy. Please check it in visual
> vm.
> >
> > In steady state, every batch has to wait for the previous 8187 batches to
> > be done before it gets a chance to be sent out, but the latency is
> counted
> > from the time is added to the queue. This is the reason that you are
> seeing
> > very high end-to-end latency.
> >
> > Have the buffer.memory to be only twice that of the batch.size so that
> > while one is in flight, you can another batch ready to go (and the
> > KafkaProducer would block to send more when there is no memory and this
> way
> > the batches are not waiting in the queue unnecessarily) . Also may be you
> > want to increase the batch.size further more, you will get even better
> > throughput with more or less same latency (as there is no shortage of
> > events in the test program).
> >
> > On Thu, Aug 13, 2015 at 1:13 PM Yuheng Du 
> > wrote:
> >
> > > Yes there is. But if we are using ProducerPerformance test, it's
> > configured
> > > as giving input when running the test command. Do you write a java
> > program
> > > to test the latency? Thanks.
> > >
> > > On Thu, Aug 13, 2015 at 3:54 PM, Alvaro Gareppe 
> > > wrote:
> > >
> > > > I'm using last one, but not using the ProducerPerformance, I created
> my
> > > > own. but I think there is a producer.properties file in config folder
> > in
> > > > kafka.. is that configuration not for this tester ?
> > > >
> > > > On Thu, Aug 13, 2015 at 4:18 PM, Yuheng Du  >
> > > > wrote:
> > > >
> > > > > Thank you Alvaro,
> > > > >
> > > > > How to use sync producers? I am running the standard
> > > ProducerPerformance
> > > > > test from kafka to measure the latency of each message to send from
> > > > > producer to broker only.
> > > > > The command is like "bin/kafka-run-class.sh
> > > > > org.apache.kafka.clients.tools.ProducerPerformance test7 5000
> 100
> > > -1
> > > > > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > > > > buffer.memory=67108864 batch.size=8196"
> > > > >
> > > > > For running producers, where should I put the producer.type=sync
> > > > > configuration into? The config/server.properties? Also Does this
> mean
> > > we
> > > > > are using batch size of 1? Which version of Kafka are you using?
> > > > > thanks.
> > > > >
> > > > > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe <
> agare...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Are you measuring latency as time between producer and consumer ?
> > > > > >
> > > > > > In that case, the ack shouldn't affect the latency, cause even
> > tough
> > > > your
> > > > > > producer is not going to wait for the ack, the consumer will only
> > get
> > > > the
> > > > > > message after its commited in the server.
> > > > > >
> > > > > > About latency my best result occur with sync producers, but the
> > > > > throughput
> > > > > > is much lower in that case.
> > > > > >
> > > > > > About not flushing to disk I'm pretty sure that it's not an
> option
> > in
> > > > > kafka
> > > > > > (correct me if I'm wrong)
> > > > > >
> > > > > > Regards,
> > > > > > Alvaro Gareppe
> > > > > >
> > > > > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du <
> > > yuheng.du.h...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Also, the latency results show no major difference when using
> > ack=0
> > > > or
> > > > > > > ack=1. Why is that?
> > > > > > >
> > > > > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
> > > > yuheng.du.h...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > I am running an experiment where 92 producers is publishing
> > data
> > > > > into 6
> > > > > > > > brokers and 10 consumer are reading online data
> simultaneously.
> > > > > > > >
> > > > > > > > How should I do to reduce the latency? Currently when I run
> the
> > > > > > producer
> > > > > > > > performance test the average latency is around 10s.
> > > > > > > >
> > > > > > > > Should I disable log.flush? How to do that? Thanks.
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > Ing. Alvaro Gareppe
> > > > > > agare...@gmail.com
> > > > > >
> > > > >
> > > >
> >

Re: Reduce latency

2015-08-17 Thread Yuheng Du
Thank you Kishore, I made the buffer twice the size of the batch size and
the latency has reduced significantly.

But is there only one thread io thread sending the batches? Can I increase
the number of threads sending the batches so more than one batch could be
sent at the same time?

Thanks.



On Thu, Aug 13, 2015 at 5:38 PM, Kishore Senji  wrote:

> Your batch.size is 8196 and your buffer.memory is 67108864. This means
> 67108864/8196
> ~ 8188 batches are in memory ready to the sent. There is only one thread io
> thread sending them. I would guess that the io thread (
> kafka-producer-network-thread) would be busy. Please check it in visual vm.
>
> In steady state, every batch has to wait for the previous 8187 batches to
> be done before it gets a chance to be sent out, but the latency is counted
> from the time is added to the queue. This is the reason that you are seeing
> very high end-to-end latency.
>
> Have the buffer.memory to be only twice that of the batch.size so that
> while one is in flight, you can another batch ready to go (and the
> KafkaProducer would block to send more when there is no memory and this way
> the batches are not waiting in the queue unnecessarily) . Also may be you
> want to increase the batch.size further more, you will get even better
> throughput with more or less same latency (as there is no shortage of
> events in the test program).
>
> On Thu, Aug 13, 2015 at 1:13 PM Yuheng Du 
> wrote:
>
> > Yes there is. But if we are using ProducerPerformance test, it's
> configured
> > as giving input when running the test command. Do you write a java
> program
> > to test the latency? Thanks.
> >
> > On Thu, Aug 13, 2015 at 3:54 PM, Alvaro Gareppe 
> > wrote:
> >
> > > I'm using last one, but not using the ProducerPerformance, I created my
> > > own. but I think there is a producer.properties file in config folder
> in
> > > kafka.. is that configuration not for this tester ?
> > >
> > > On Thu, Aug 13, 2015 at 4:18 PM, Yuheng Du 
> > > wrote:
> > >
> > > > Thank you Alvaro,
> > > >
> > > > How to use sync producers? I am running the standard
> > ProducerPerformance
> > > > test from kafka to measure the latency of each message to send from
> > > > producer to broker only.
> > > > The command is like "bin/kafka-run-class.sh
> > > > org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100
> > -1
> > > > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > > > buffer.memory=67108864 batch.size=8196"
> > > >
> > > > For running producers, where should I put the producer.type=sync
> > > > configuration into? The config/server.properties? Also Does this mean
> > we
> > > > are using batch size of 1? Which version of Kafka are you using?
> > > > thanks.
> > > >
> > > > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe 
> > > > wrote:
> > > >
> > > > > Are you measuring latency as time between producer and consumer ?
> > > > >
> > > > > In that case, the ack shouldn't affect the latency, cause even
> tough
> > > your
> > > > > producer is not going to wait for the ack, the consumer will only
> get
> > > the
> > > > > message after its commited in the server.
> > > > >
> > > > > About latency my best result occur with sync producers, but the
> > > > throughput
> > > > > is much lower in that case.
> > > > >
> > > > > About not flushing to disk I'm pretty sure that it's not an option
> in
> > > > kafka
> > > > > (correct me if I'm wrong)
> > > > >
> > > > > Regards,
> > > > > Alvaro Gareppe
> > > > >
> > > > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du <
> > yuheng.du.h...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Also, the latency results show no major difference when using
> ack=0
> > > or
> > > > > > ack=1. Why is that?
> > > > > >
> > > > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
> > > yuheng.du.h...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > I am running an experiment where 92 producers is publishing
> data
> > > > into 6
> > > > > > > brokers and 10 consumer are reading online data simultaneously.
> > > > > > >
> > > > > > > How should I do to reduce the latency? Currently when I run the
> > > > > producer
> > > > > > > performance test the average latency is around 10s.
> > > > > > >
> > > > > > > Should I disable log.flush? How to do that? Thanks.
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Ing. Alvaro Gareppe
> > > > > agare...@gmail.com
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Ing. Alvaro Gareppe
> > > agare...@gmail.com
> > >
> >
>


Re: Reduce latency

2015-08-13 Thread Kishore Senji
Your batch.size is 8196 and your buffer.memory is 67108864. This means
67108864/8196
~ 8188 batches are in memory ready to the sent. There is only one thread io
thread sending them. I would guess that the io thread (
kafka-producer-network-thread) would be busy. Please check it in visual vm.

In steady state, every batch has to wait for the previous 8187 batches to
be done before it gets a chance to be sent out, but the latency is counted
from the time is added to the queue. This is the reason that you are seeing
very high end-to-end latency.

Have the buffer.memory to be only twice that of the batch.size so that
while one is in flight, you can another batch ready to go (and the
KafkaProducer would block to send more when there is no memory and this way
the batches are not waiting in the queue unnecessarily) . Also may be you
want to increase the batch.size further more, you will get even better
throughput with more or less same latency (as there is no shortage of
events in the test program).

On Thu, Aug 13, 2015 at 1:13 PM Yuheng Du  wrote:

> Yes there is. But if we are using ProducerPerformance test, it's configured
> as giving input when running the test command. Do you write a java program
> to test the latency? Thanks.
>
> On Thu, Aug 13, 2015 at 3:54 PM, Alvaro Gareppe 
> wrote:
>
> > I'm using last one, but not using the ProducerPerformance, I created my
> > own. but I think there is a producer.properties file in config folder in
> > kafka.. is that configuration not for this tester ?
> >
> > On Thu, Aug 13, 2015 at 4:18 PM, Yuheng Du 
> > wrote:
> >
> > > Thank you Alvaro,
> > >
> > > How to use sync producers? I am running the standard
> ProducerPerformance
> > > test from kafka to measure the latency of each message to send from
> > > producer to broker only.
> > > The command is like "bin/kafka-run-class.sh
> > > org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100
> -1
> > > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > > buffer.memory=67108864 batch.size=8196"
> > >
> > > For running producers, where should I put the producer.type=sync
> > > configuration into? The config/server.properties? Also Does this mean
> we
> > > are using batch size of 1? Which version of Kafka are you using?
> > > thanks.
> > >
> > > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe 
> > > wrote:
> > >
> > > > Are you measuring latency as time between producer and consumer ?
> > > >
> > > > In that case, the ack shouldn't affect the latency, cause even tough
> > your
> > > > producer is not going to wait for the ack, the consumer will only get
> > the
> > > > message after its commited in the server.
> > > >
> > > > About latency my best result occur with sync producers, but the
> > > throughput
> > > > is much lower in that case.
> > > >
> > > > About not flushing to disk I'm pretty sure that it's not an option in
> > > kafka
> > > > (correct me if I'm wrong)
> > > >
> > > > Regards,
> > > > Alvaro Gareppe
> > > >
> > > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du <
> yuheng.du.h...@gmail.com>
> > > > wrote:
> > > >
> > > > > Also, the latency results show no major difference when using ack=0
> > or
> > > > > ack=1. Why is that?
> > > > >
> > > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
> > yuheng.du.h...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > I am running an experiment where 92 producers is publishing data
> > > into 6
> > > > > > brokers and 10 consumer are reading online data simultaneously.
> > > > > >
> > > > > > How should I do to reduce the latency? Currently when I run the
> > > > producer
> > > > > > performance test the average latency is around 10s.
> > > > > >
> > > > > > Should I disable log.flush? How to do that? Thanks.
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Ing. Alvaro Gareppe
> > > > agare...@gmail.com
> > > >
> > >
> >
> >
> >
> > --
> > Ing. Alvaro Gareppe
> > agare...@gmail.com
> >
>


Re: Reduce latency

2015-08-13 Thread Yuheng Du
Yes there is. But if we are using ProducerPerformance test, it's configured
as giving input when running the test command. Do you write a java program
to test the latency? Thanks.

On Thu, Aug 13, 2015 at 3:54 PM, Alvaro Gareppe  wrote:

> I'm using last one, but not using the ProducerPerformance, I created my
> own. but I think there is a producer.properties file in config folder in
> kafka.. is that configuration not for this tester ?
>
> On Thu, Aug 13, 2015 at 4:18 PM, Yuheng Du 
> wrote:
>
> > Thank you Alvaro,
> >
> > How to use sync producers? I am running the standard ProducerPerformance
> > test from kafka to measure the latency of each message to send from
> > producer to broker only.
> > The command is like "bin/kafka-run-class.sh
> > org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100 -1
> > acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> > buffer.memory=67108864 batch.size=8196"
> >
> > For running producers, where should I put the producer.type=sync
> > configuration into? The config/server.properties? Also Does this mean we
> > are using batch size of 1? Which version of Kafka are you using?
> > thanks.
> >
> > On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe 
> > wrote:
> >
> > > Are you measuring latency as time between producer and consumer ?
> > >
> > > In that case, the ack shouldn't affect the latency, cause even tough
> your
> > > producer is not going to wait for the ack, the consumer will only get
> the
> > > message after its commited in the server.
> > >
> > > About latency my best result occur with sync producers, but the
> > throughput
> > > is much lower in that case.
> > >
> > > About not flushing to disk I'm pretty sure that it's not an option in
> > kafka
> > > (correct me if I'm wrong)
> > >
> > > Regards,
> > > Alvaro Gareppe
> > >
> > > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du 
> > > wrote:
> > >
> > > > Also, the latency results show no major difference when using ack=0
> or
> > > > ack=1. Why is that?
> > > >
> > > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du <
> yuheng.du.h...@gmail.com>
> > > > wrote:
> > > >
> > > > > I am running an experiment where 92 producers is publishing data
> > into 6
> > > > > brokers and 10 consumer are reading online data simultaneously.
> > > > >
> > > > > How should I do to reduce the latency? Currently when I run the
> > > producer
> > > > > performance test the average latency is around 10s.
> > > > >
> > > > > Should I disable log.flush? How to do that? Thanks.
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Ing. Alvaro Gareppe
> > > agare...@gmail.com
> > >
> >
>
>
>
> --
> Ing. Alvaro Gareppe
> agare...@gmail.com
>


Re: Reduce latency

2015-08-13 Thread Alvaro Gareppe
I'm using last one, but not using the ProducerPerformance, I created my
own. but I think there is a producer.properties file in config folder in
kafka.. is that configuration not for this tester ?

On Thu, Aug 13, 2015 at 4:18 PM, Yuheng Du  wrote:

> Thank you Alvaro,
>
> How to use sync producers? I am running the standard ProducerPerformance
> test from kafka to measure the latency of each message to send from
> producer to broker only.
> The command is like "bin/kafka-run-class.sh
> org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100 -1
> acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
> buffer.memory=67108864 batch.size=8196"
>
> For running producers, where should I put the producer.type=sync
> configuration into? The config/server.properties? Also Does this mean we
> are using batch size of 1? Which version of Kafka are you using?
> thanks.
>
> On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe 
> wrote:
>
> > Are you measuring latency as time between producer and consumer ?
> >
> > In that case, the ack shouldn't affect the latency, cause even tough your
> > producer is not going to wait for the ack, the consumer will only get the
> > message after its commited in the server.
> >
> > About latency my best result occur with sync producers, but the
> throughput
> > is much lower in that case.
> >
> > About not flushing to disk I'm pretty sure that it's not an option in
> kafka
> > (correct me if I'm wrong)
> >
> > Regards,
> > Alvaro Gareppe
> >
> > On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du 
> > wrote:
> >
> > > Also, the latency results show no major difference when using ack=0 or
> > > ack=1. Why is that?
> > >
> > > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du 
> > > wrote:
> > >
> > > > I am running an experiment where 92 producers is publishing data
> into 6
> > > > brokers and 10 consumer are reading online data simultaneously.
> > > >
> > > > How should I do to reduce the latency? Currently when I run the
> > producer
> > > > performance test the average latency is around 10s.
> > > >
> > > > Should I disable log.flush? How to do that? Thanks.
> > > >
> > >
> >
> >
> >
> > --
> > Ing. Alvaro Gareppe
> > agare...@gmail.com
> >
>



-- 
Ing. Alvaro Gareppe
agare...@gmail.com


Re: Reduce latency

2015-08-13 Thread Yuheng Du
Thank you Alvaro,

How to use sync producers? I am running the standard ProducerPerformance
test from kafka to measure the latency of each message to send from
producer to broker only.
The command is like "bin/kafka-run-class.sh
org.apache.kafka.clients.tools.ProducerPerformance test7 5000 100 -1
acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
buffer.memory=67108864 batch.size=8196"

For running producers, where should I put the producer.type=sync
configuration into? The config/server.properties? Also Does this mean we
are using batch size of 1? Which version of Kafka are you using?
thanks.

On Thu, Aug 13, 2015 at 3:01 PM, Alvaro Gareppe  wrote:

> Are you measuring latency as time between producer and consumer ?
>
> In that case, the ack shouldn't affect the latency, cause even tough your
> producer is not going to wait for the ack, the consumer will only get the
> message after its commited in the server.
>
> About latency my best result occur with sync producers, but the throughput
> is much lower in that case.
>
> About not flushing to disk I'm pretty sure that it's not an option in kafka
> (correct me if I'm wrong)
>
> Regards,
> Alvaro Gareppe
>
> On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du 
> wrote:
>
> > Also, the latency results show no major difference when using ack=0 or
> > ack=1. Why is that?
> >
> > On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du 
> > wrote:
> >
> > > I am running an experiment where 92 producers is publishing data into 6
> > > brokers and 10 consumer are reading online data simultaneously.
> > >
> > > How should I do to reduce the latency? Currently when I run the
> producer
> > > performance test the average latency is around 10s.
> > >
> > > Should I disable log.flush? How to do that? Thanks.
> > >
> >
>
>
>
> --
> Ing. Alvaro Gareppe
> agare...@gmail.com
>


Re: Reduce latency

2015-08-13 Thread Alvaro Gareppe
Are you measuring latency as time between producer and consumer ?

In that case, the ack shouldn't affect the latency, cause even tough your
producer is not going to wait for the ack, the consumer will only get the
message after its commited in the server.

About latency my best result occur with sync producers, but the throughput
is much lower in that case.

About not flushing to disk I'm pretty sure that it's not an option in kafka
(correct me if I'm wrong)

Regards,
Alvaro Gareppe

On Thu, Aug 13, 2015 at 12:59 PM, Yuheng Du 
wrote:

> Also, the latency results show no major difference when using ack=0 or
> ack=1. Why is that?
>
> On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du 
> wrote:
>
> > I am running an experiment where 92 producers is publishing data into 6
> > brokers and 10 consumer are reading online data simultaneously.
> >
> > How should I do to reduce the latency? Currently when I run the producer
> > performance test the average latency is around 10s.
> >
> > Should I disable log.flush? How to do that? Thanks.
> >
>



-- 
Ing. Alvaro Gareppe
agare...@gmail.com


Reduce latency

2015-08-13 Thread Yuheng Du
I am running an experiment where 92 producers is publishing data into 6
brokers and 10 consumer are reading online data simultaneously.

How should I do to reduce the latency? Currently when I run the producer
performance test the average latency is around 10s.

Should I disable log.flush? How to do that? Thanks.


Re: Reduce latency

2015-08-13 Thread Yuheng Du
Also, the latency results show no major difference when using ack=0 or
ack=1. Why is that?

On Thu, Aug 13, 2015 at 11:51 AM, Yuheng Du 
wrote:

> I am running an experiment where 92 producers is publishing data into 6
> brokers and 10 consumer are reading online data simultaneously.
>
> How should I do to reduce the latency? Currently when I run the producer
> performance test the average latency is around 10s.
>
> Should I disable log.flush? How to do that? Thanks.
>