+1 I think there is some pain with changing any default, but this is well worth it.
The reality is that the profile of Kafka usage has changed significantly since replication was added to Kafka and these defaults were chosen. At that time usage was primarily high volume event and log data and the performance of acks=all was quite bad. Both of those two things have changed. Now I think we see far more critical production uses, and the cost of the better semantics is really pretty minimal. Kafka has very strong guarantees now to support that type of use case but you really have to understand the knobs and go in and set them correctly. A good example of this is Gwen's talk on tuning Kafka for reliability. Basically that 45 minute talk is the least amount you need to know to not risk your data. Needless to say we see people goofing things up and being unhappy about it. The reality is that our performance in the "safe" mode is actually quite spectacular. Back of the envelope it looks like about 570k msg/sec for 128 byte messages for acks=all vs 680k msgs/sec for acks=1. I suspect with tuning we could shrink that, but even as is I think the number of apps that need throughput north of 500k msgs/sec/instance is small. The only thing I'd add is that let's make sure the out-of-the box usage on your own machine isn't complicated (i.e. not introduce any setting that has to be adjusted to run in single node configuration) as a lot of people depend on that. -Jay On Tue, Aug 8, 2017 at 5:23 PM Apurva Mehta <apu...@confluent.io> wrote: > Hi, > > I've put together a new KIP which proposes to ship Kafka with its strongest > delivery guarantees by default. > > We currently ship with at most once semantics and don't provide any > ordering guarantees per partition. The proposal is is to provide exactly > once in order delivery per partition by default in the upcoming 1.0.0 > release. > > The KIP linked to below also outlines the performance characteristics of > the proposed default. > > The KIP is here: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 185%3A+Make+exactly+once+in+order+delivery+per+partition+ > the+default+producer+setting > > Please have a look, I would love your feedback! > > Thanks, > Apurva >