This is a good question.

As mentioned we have some experience running this with no ack and there are
a lot of downsides. We considered making the ack optional, but this would
complicate the producer api since we could give back the offset only in the
case where there is an ack.

Thinking about it more we realized there is no real performance hit, just
latency, and you only pay for the latency if you want to wait for the
response. This resulted in the current tentative plan which is to make all
requests async, always return "future response" so you only block if you
want to get the result. This is the best possible end-state since we can
give the rich general api with the same performance as without the ack.

However this requires a fairly large change in the client which we haven't
done yet.

So in 0.8 synchronous producer performance will decrease. Asynchronous
production should probably not be too much worse because the async
production and batching masks and amortizes the latency. It is too early to
say how much worse it will be as there are still a few perf issues to
resolve.

It would be reasonable if people were a little annoyed by this since we are
effectively making the software worse on some dimensions before we make it
better. Our reasoning was that batching up even more changes into a single
release was just too dangerous. People who care about replication will
(hopefully) care enough about this that taking a hit on sync producer
performance will be okay, and people who don't care about replication can
just skip a version since that is the major feature in 0.8.

-Jay


On Tue, Jan 22, 2013 at 7:10 PM, S Ahmed <sahmed1...@gmail.com> wrote:

> Neha,
>
> I see, so that is a fairly substantial change, ofcourse it has its
> advantage of guaranteeing a higher degree of durability but as a
> significant cost (round trip that the consumer has to wait for).  I know
> someone mentioned creating a asych. consumer with a future.
>
> Do you have a 'gut' feeling performance will be the same as in .7 or x%
> slower?  (or you have no idea as of yet as you guys are still going to work
> on perf)
>
>
> On Fri, Jan 18, 2013 at 8:42 PM, Neha Narkhede <neha.narkh...@gmail.com
> >wrote:
>
> > >> producer.num.acks=0
> >
> > There is still a difference between the 0.7 and 0.8 Kafka behavior in the
> > sense that in 0.7, the producer fired away requests at the broker without
> > waiting for an ack. In 0.8, even with num.acks=0, the producer writes are
> > going to be synchronous and it won't be able to send the next request
> until
> > the ack for the previous one comes back.
> >
> > Thanks,
> > Neha
> >
> >
> > On Fri, Jan 18, 2013 at 12:24 PM, S Ahmed <sahmed1...@gmail.com> wrote:
> >
> > > I see ok, so if you wanted to compare .7 with .8 on the same footing,
> > then
> > > you would set it to 0 right? (since 0.7 is fire and forget)
> > >
> > > producer.num.acks=0
> > >
> > >
> > > On Thu, Jan 17, 2013 at 11:45 PM, Jun Rao <jun...@gmail.com> wrote:
> > >
> > > > I means wait for the data reaches all replicas (that are in sync).
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Thu, Jan 17, 2013 at 6:42 PM, S Ahmed <sahmed1...@gmail.com>
> wrote:
> > > >
> > > > > producer.num.acks=-1 means what sorry? is it that all replica's are
> > > > written
> > > > > too?
> > > > >
> > > > >
> > > > > On Thu, Jan 17, 2013 at 12:09 PM, Neha Narkhede <
> > > neha.narkh...@gmail.com
> > > > > >wrote:
> > > > >
> > > > > > Looks like Jun's email didn't format the output properly. I've
> > > > published
> > > > > > some preliminary producer throughput performance numbers on our
> > > > > performance
> > > > > > wiki -
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Performance+testing#Performancetesting-Producerthroughput
> > > > > >
> > > > > > These tests measure producer throughput in the worst case
> scenario
> > > > > > (producer.num.acks=-1) i.e. max durability setting. The baseline
> > with
> > > > 0.7
> > > > > > would be to compare producer throughput with num.acks=0. We are
> > > working
> > > > > on
> > > > > > those tests now.
> > > > > >
> > > > > > Thanks,
> > > > > > Neha
> > > > > >
> > > > > >
> > > > > > On Thu, Jan 17, 2013 at 8:43 AM, Jun Rao <jun...@gmail.com>
> wrote:
> > > > > >
> > > > > > > We also did some perf test on 0.8 using the following command.
> > All
> > > > > > configs
> > > > > > > on the broker are the defaults.
> > > > > > > bin/kafka-run-class.sh kafka.perf.ProducerPerformance
> > --broker-list
> > > > > > > localhost:9092 --initial-message-id 0 --messages 2000000
> --topics
> > > > > > topic_001
> > > > > > > --request-num-acks -1 --batch-size 100 --threads 1
> --message-size
> > > > 1024
> > > > > > > --compression-codec 0
> > > > > > >
> > > > > > > The following is our preliminary result. Could you try this on
> > your
> > > > > > > environment? For replication factor larger than 1, we will try
> > > ack=1
> > > > > and
> > > > > > > report the numbers later. It should provide better throughput.
> > > > Thanks,
> > > > > > >
> > > > > > > *No. of Brokers = 1 / Replication Factor = 1 (Partition =
> > > > 1)**Producer
> > > > > > > threads**comp**msg size**Acks**batch**Thru Put
> > > > > > > (MB/s)*101024-115.49201024-11
> > > > > > >
> > > > >
> > >
> 9.38501024-1116.611001024-1119.54101024-15025.72201024-15039.25501024-150
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 54.171001024-15056.71101024-110027.97201024-110045.05501024-110058.011001024
> > > > > > > -110059.82*No. of Brokers = 2 / Replication Factor = 2
> > (Partitions
> > > =
> > > > > > > 1)**Producer
> > > > > > > threads**comp**msg size**Acks**batch**Thru Put
> > > > > > > (MB/s)*101024-110.58201024-11
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 1.17501024-111.601001024-113.15101024-1507.48201024-15013.89501024-15018.11
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 1001024-15020.91101024-11008.72201024-110016.84501024-110020.661001024-1100
> > > > > > > 23.82*No. of Brokers = 3 / Replication Factor = 3 (Partitions =
> > > > > > > 1)**Producer
> > > > > > > threads**comp**msg size**Acks**batch**Thru Put
> > > > > > > (MB/s)*101024-110.53201024-11
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 0.94501024-111.721001024-112.78101024-1507.08201024-15013.40501024-15018.11
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 1001024-15021.01101024-11008.09201024-110014.88501024-110019.931001024-1100
> > > > > > > 23.22
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Wed, Jan 16, 2013 at 8:33 PM, Jun Guo -X (jungu - CIIC at
> > > Cisco) <
> > > > > > > ju...@cisco.com> wrote:
> > > > > > >
> > > > > > > >  Hi,****
> > > > > > > >
> > > > > > > >       I do producer(Kafka 0.8) throughput test many times.
> But
> > > the
> > > > > > > > average value is 3MB/S. Below is my test environment:****
> > > > > > > >
> > > > > > > >        CPU core      :16 ****
> > > > > > > >
> > > > > > > >        Vendor_id     :GenuineIntel****
> > > > > > > >
> > > > > > > >        Cpu family     :6****
> > > > > > > >
> > > > > > > >        Cpu MHz      :2899.999****
> > > > > > > >
> > > > > > > >        Cache size    :20480 KB****
> > > > > > > >
> > > > > > > >        Cpu level      :13****
> > > > > > > >
> > > > > > > >        MEM             :16330832KB=15.57GB****
> > > > > > > >
> > > > > > > >        Disk       : RAID5****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > >        I don’t know the detail information about the disk,
> such
> > > as
> > > > > > > > rotation. But I do some test about the I/O performance of the
> > > disk.
> > > > > The
> > > > > > > > write rate is 500MB~600MB/S, the read rate is 180MB/S. The
> > detail
> > > > is
> > > > > as
> > > > > > > > below. ****
> > > > > > > >
> > > > > > > > [image: cid:image002.png@01CDF4AE.52046900]****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > And I adjust the broker configuration file as the official
> > > document
> > > > > > says
> > > > > > > > as below. And I adjust the JVM to 5120MB. ****
> > > > > > > >
> > > > > > > > I run producer performance test with the script
> > > > > > > > kafka-producer-perf-test.sh, with the test command is ****
> > > > > > > >
> > > > > > > > *bin/kafka-producer-perf-test.sh --broker-list
> > > 10.75.167.46:49092
> > > > > > --topics
> > > > > > >
> topic_perf_46_1,topic_perf_46_2,topic_perf_46_3,topic_perf_46_4,
> > > > > > > > topic_perf_46_5,topic_perf_46_6,
> > > > > > > >
> > topic_perf_46_7,topic_perf_46_8,topic_perf_46_9,topic_perf_46_10
> > > > > > > > --initial-message-id 0 --threads 200 --messages 1000000
> > > > > --message-size
> > > > > > > 200
> > > > > > > > --compression-codec 1*
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > But the test result is also not as good as the official
> > document
> > > > > > > > says(50MB/S, and that value in your paper is 100MB/S). The
> test
> > > > > result
> > > > > > is
> > > > > > > > as below:****
> > > > > > > >
> > > > > > > > 2013-01-17 04:15:24:768, 2013-01-17 04:25:01:637, 0, 200,
> 200,
> > > > > > 1907.35, *
> > > > > > > > 3.3064,* 10000000, 17334.9582****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > On the other hand, I do consumer throughput test, the result
> is
> > > > about
> > > > > > > > 60MB/S while that value in official document is 100MB/S.****
> > > > > > > >
> > > > > > > > I really don’t know why?****
> > > > > > > >
> > > > > > > > You know high throughput is one of the most important
> features
> > of
> > > > > > Kafka.
> > > > > > > > So I am really concerned with it.****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > Thanks and best regards!****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > *From:* Jay Kreps [mailto:jkr...@linkedin.com]
> > > > > > > > *Sent:* 2013年1月16日 2:22
> > > > > > > > *To:* Jun Guo -X (jungu - CIIC at Cisco)
> > > > > > > > *Subject:* RE: About acknowledge from broker to producer in
> > your
> > > > > > > paper.***
> > > > > > > > *
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > Not sure which version you are using... ****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > In 0.7 this would happen only if there was a socket level
> error
> > > > (i.e.
> > > > > > > > can't connect to the host). This covers a lot of cases since
> in
> > > the
> > > > > > event
> > > > > > > > of I/O errors (disk full, etc) we just have that node shut
> > itself
> > > > > down
> > > > > > to
> > > > > > > > let others take over.****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > In 0.8 we send all errors back to the client.****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > So the difference is that, for example, in the event of a
> disk
> > > > error,
> > > > > > in
> > > > > > > > 0.7 the client would send a message, the broker would get an
> > > error
> > > > > and
> > > > > > > > shoot itself in the head and disconnect its clients, and the
> > > client
> > > > > > would
> > > > > > > > get the error the next time it tried to send a message. So in
> > 0.7
> > > > the
> > > > > > > error
> > > > > > > > might not get passed back to the client until the second
> > message
> > > > > send.
> > > > > > In
> > > > > > > > 0.8 this would happen with the first send, which is an
> > > > > improvement.****
> > > > > > > >
> > > > > > > > ** **
> > > > > > > >
> > > > > > > > -Jay****
> > > > > > > >  ------------------------------
> > > > > > > >
> > > > > > > > *From:* Jun Guo -X (jungu - CIIC at Cisco) [ju...@cisco.com]
> > > > > > > > *Sent:* Monday, January 14, 2013 9:45 PM
> > > > > > > > *To:* Jay Kreps
> > > > > > > > *Subject:* About acknowledge from broker to producer in your
> > > > > paper.****
> > > > > > > >
> > > > > > > > Hi,****
> > > > > > > >
> > > > > > > >        I have read your paper *Kafka: a Distributed Messaging
> > > > System
> > > > > > for
> > > > > > > > Log Processing* .****
> > > > > > > >
> > > > > > > >        In experimental results part. There are some words as
> > > > > below:****
> > > > > > > >
> > > > > > > >        ****
> > > > > > > >
> > > > > > > >        *There are a few reasons why Kafka performed much
> > better.
> > > > > First,
> > > > > > > > the Kafka producer currently doesn**’t wait for
> > acknowledgements
> > > > from
> > > > > > the
> > > > > > > > broker and sends messages as faster as the broker can handle.
> > > This
> > > > > > > > significantly increased the throughput of the publisher.
> With a
> > > > batch
> > > > > > > size
> > > > > > > > of 50, a single Kafka producer almost saturated the 1Gb link
> > > > between
> > > > > > the
> > > > > > > > producer and the broker. This is a valid optimization for the
> > log
> > > > > > > > aggregation case, as data must be sent asynchronously to
> avoid
> > > > > > > introducing
> > > > > > > > any latency into the live serving of traffic. We note that
> > > without
> > > > > > > > acknowledging the producer, there is no guarantee that every
> > > > > published
> > > > > > > > message is actually received by the broker. For many types of
> > log
> > > > > data,
> > > > > > > it
> > > > > > > > is desirable to trade durability for throughput, as long as
> the
> > > > > number
> > > > > > of
> > > > > > > > dropped messages is relatively small. However, we do plan
> > to*****
> > > > > > > >
> > > > > > > > *address the durability issue for more critical data in the
> > > > > > future.*****
> > > > > > > >
> > > > > > > >  ****
> > > > > > > >
> > > > > > > >        But I have done a series of test. I found that ,if I
> > shut
> > > > down
> > > > > > all
> > > > > > > > the brokers, when I send a message from producer to broker,
> the
> > > > > > producer
> > > > > > > > will report kafka.common.FailedToSendMessageException . It
> > says,
> > > > > Failed
> > > > > > > to
> > > > > > > > send messages after 3 tries.****
> > > > > > > >
> > > > > > > > ****
> > > > > > > >
> > > > > > > >        If there is no acknowledge from broker, how the
> producer
> > > > find
> > > > > > the
> > > > > > > > sending is failed? And how it try 3 times?****
> > > > > > > >
> > > > > > > >  ****
> > > > > > > >
> > > > > > > >        Maybe, the acknowledge in your paper refers to another
> > > > thing,
> > > > > if
> > > > > > > so
> > > > > > > > ,please tell what is the meaning of acknowledge?****
> > > > > > > >
> > > > > > > >  ****
> > > > > > > >
> > > > > > > >        Many thanks and best regards!****
> > > > > > > >
> > > > > > > >  ****
> > > > > > > >
> > > > > > > > Guo Jun****
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to