I would be in favor of defaulting acks=all. I have found that most people want to start with the stronger/safer guarantees and then adjust them for performance on a case by case basis. This gives them a chance to understand and accept the tradeoffs.
A few other defaults I would be in favor of changing (some are harder and more controversial than others) are: Broker: - zookeeper.chroot=kafka (was "") - This will be easiest when direct communication to zookeeper isn't done by clients Producer: - block.on.buffer.full=true (was false) - max.in.flight.requests.per.connection=1 (was 5) All: - *receive.buffer.bytes=-1 (was 102400) - *send.buffer.bytes=-1 (was 102400) On Fri, Feb 3, 2017 at 2:03 AM, Ismael Juma <ism...@juma.me.uk> wrote: > I'd be in favour too. > > Ismael > > On 3 Feb 2017 7:33 am, "Ewen Cheslack-Postava" <e...@confluent.io> wrote: > > > On Thu, Feb 2, 2017 at 11:21 PM, James Cheng <wushuja...@gmail.com> > wrote: > > > > > Ewen, > > > > > > Ah right, that's a good point. > > > > > > My initial reaction to your examples was that "well, those should be in > > > separate topics", but then I realized that people choose their topics > > for a > > > variety of reasons. Sometimes they organize it based on their > producers, > > > sometimes they organize it based on the nature of the data, but > sometimes > > > (as you gave examples about), they may organize it based on the > consuming > > > application. And there are valid reason to want different data types > in a > > > single topic: > > > > > > 1) You get global ordering > > > 2) You get persistent ordering in the case of re-reads (where as > reading > > 2 > > > topics would cause different ordering upon re-reads.) > > > 3) Logically-related data types all co-located. > > > > > > I do still think it'd be convenient to only have to set > > > min.insync.replicas on a topic and not have to require producing > > > applications to also set acks=all. It'd then be a single thing you have > > to > > > configure, instead of the current 2 things. (since, as currently > > > implemented, you have to set both things, in order to achieve high > > > durability.) > > > > > > > I entirely agree, I think the default should be acks=all and then this > > would be true :) Similar to the unclean leader election setting, I think > > defaulting to durable by default is a better choice. I understand > > historically why a different choice was made (Kafka didn't start out as a > > replicated, durable storage system), but given how it has evolved I think > > durable by default would be a better choice on both the broker and > > producer. > > > > > > > > > > But I admit that it's hard to find the balance of features/simplicity/ > > complexity, > > > to handle all the use cases. > > > > > > > Perhaps the KIP-106 adjustment to unclean leader election could benefit > > from a sister KIP for adjusting the default producer acks setting? > > > > Not sure how popular it would be, but I would be in favor. > > > > -Ewen > > > > > > > > > > Thanks, > > > -James > > > > > > > On Feb 2, 2017, at 9:42 PM, Ewen Cheslack-Postava <e...@confluent.io > > > > > wrote: > > > > > > > > James, > > > > > > > > Great question, I probably should have been clearer. log data is an > > > example > > > > where the app (or even instance of the app) might know best what the > > > right > > > > tradeoff is. Depending on your strategy for managing logs, you may or > > may > > > > not be mixing multiple logs (and logs from different deployments) > into > > > the > > > > same topic. For example, if you key by application, then you have an > > easy > > > > way to split logs up while still getting a global feed of log > messages. > > > > Maybe logs from one app are really critical and we want to retry, but > > > from > > > > another app are just a nice to have. > > > > > > > > There are other examples even within a single app. For example, a > > gaming > > > > company might report data from a user of a game to the same topic but > > > want > > > > 2 producers with different reliability levels (and possibly where the > > > > ordering constraints across the two sets that might otherwise cause > you > > > to > > > > use a single consumer are not an issue). High frequency telemetry on > a > > > > player might be desirable to have, but not the end of the world if > some > > > is > > > > lost. In contrast, they may want a stronger guarantee for, e.g., > > sometime > > > > like chat messages, where they want to have a permanent record of > them > > in > > > > all circumstances. > > > > > > > > -Ewen > > > > > > > > On Fri, Jan 27, 2017 at 12:59 AM, James Cheng <wushuja...@gmail.com> > > > wrote: > > > > > > > >> > > > >>> On Jan 27, 2017, at 12:18 AM, Ewen Cheslack-Postava < > > e...@confluent.io > > > > > > > >> wrote: > > > >>> > > > >>> On Thu, Jan 26, 2017 at 4:23 PM, Luciano Afranllie < > > > >> listas.luaf...@gmail.com > > > >>>> wrote: > > > >>> > > > >>>> I was thinking about the situation where you have less brokers in > > the > > > >> ISR > > > >>>> list than the number set in min.insync.replicas. > > > >>>> > > > >>>> My idea was that if I, as an administrator, for a given topic, > want > > to > > > >>>> favor durability over availability, then if that topic has less > ISR > > > than > > > >>>> the value set in min.insync.replicas I may want to stop producing > to > > > the > > > >>>> topic. In the way min.insync.replicas and ack work, I need to > > > coordinate > > > >>>> with all producers in order to achieve this. There is no way (or I > > > don't > > > >>>> know it) to globally enforce stop producing to a topic if it is > > under > > > >>>> replicated. > > > >>>> > > > >>>> I don't see why, for the same topic, some producers might want get > > an > > > >> error > > > >>>> when the number of ISR is below min.insync.replicas while other > > > >> producers > > > >>>> don't. I think it could be more useful to be able to set that ALL > > > >> producers > > > >>>> should get an error when a given topic is under replicated so they > > > stop > > > >>>> producing, than for a single producer to get an error when ANY > topic > > > is > > > >>>> under replicated. I don't have a lot of experience with Kafka so I > > may > > > >> be > > > >>>> missing some use cases. > > > >>>> > > > >>> > > > >>> It's also a matter of not having to do a ton of configuration on a > > > >>> per-topic basis. Putting some control in the producer apps hands > > means > > > >> you > > > >>> can set reasonably global defaults which make sense for apps that > > > require > > > >>> stronger durability while letting cases that have lower > requirements > > > >> still > > > >>> benefit from the durability before consumers see data but not block > > > >>> producers because the producer chooses lower requirements. WIthout > > > >>> requiring the ability to make config changes on the Kafka brokers > > > (which > > > >>> may be locked down and restricted only to Kafka admins), the > producer > > > >>> application can choose to accept weaker guarantees based on the > > > tradeoffs > > > >>> it needs to make. > > > >>> > > > >> > > > >> I'm not sure I follow, Ewen. > > > >> > > > >> I do agree that if I set min.insync.replicas at a broker level, then > > of > > > >> course I would like individual producers to decide whether their > topic > > > >> (which inherits from the global setting) should reject writes if > that > > > topic > > > >> has size(ISR)<min.insync.replicas. > > > >> > > > >> But on a topic-level... are you saying that if a particular topic > has > > > >> min.insync.replicas set, that you want producers to have the > > > flexibility to > > > >> decide on whether they want durability vs availability? > > > >> > > > >> Often times (but not always), a particular topic is used only by a > > small > > > >> set of producers with a specific set of data. The durability > settings > > > would > > > >> usually be chosen due to the nature of the data, rather than based > on > > > who > > > >> produced the data, and so it makes sense to me that the durability > > > should > > > >> be on the entire topic, not by the producer. > > > >> > > > >> What is a use case where you have multiple producers writing to the > > same > > > >> topic but would want different durability? > > > >> > > > >> -James > > > >> > > > >>> The ability to make this tradeoff in different places can seem more > > > >> complex > > > >>> (and really by definition *is* more complex), but it also offers > more > > > >>> flexibility. > > > >>> > > > >>> -Ewen > > > >>> > > > >>> > > > >>>> But I understand your point, min.insync.replicas setting should be > > > >>>> understood as "if a producer wants to get an error when topics are > > > under > > > >>>> replicated, then how many replicas are enough for not raising an > > > error?" > > > >>>> > > > >>>> > > > >>>> On Thu, Jan 26, 2017 at 4:16 PM, Ewen Cheslack-Postava < > > > >> e...@confluent.io> > > > >>>> wrote: > > > >>>> > > > >>>>> The acks setting for the producer doesn't affect the final > > durability > > > >>>>> guarantees. These are still enforced by the replication and min > ISR > > > >>>>> settings. Instead, the ack setting just lets the producer control > > how > > > >>>>> durable the write is before *that producer* can consider the > write > > > >>>>> "complete", i.e. before it gets an ack. > > > >>>>> > > > >>>>> -Ewen > > > >>>>> > > > >>>>> On Tue, Jan 24, 2017 at 12:46 PM, Luciano Afranllie < > > > >>>>> listas.luaf...@gmail.com> wrote: > > > >>>>> > > > >>>>>> Hi everybody > > > >>>>>> > > > >>>>>> I am trying to understand why Kafka let each individual > producer, > > > on a > > > >>>>>> connection per connection basis, choose the tradeoff between > > > >>>> availability > > > >>>>>> and durability, honoring min.insync.replicas value only if > > producer > > > >>>> uses > > > >>>>>> ack=all. > > > >>>>>> > > > >>>>>> I mean, for a single topic, cluster administrators can't enforce > > > >>>> messages > > > >>>>>> to be stores in a minimum number of replicas without > coordinating > > > with > > > >>>>> all > > > >>>>>> producers to that topic so all of them use ack=all. > > > >>>>>> > > > >>>>>> Is there something that I am missing? Is there any other > strategy > > to > > > >>>>>> overcome this situation? > > > >>>>>> > > > >>>>>> Regards > > > >>>>>> Luciano > > > >>>>>> > > > >>>>> > > > >>>> > > > >> > > > >> > > > > > > > > > -- Grant Henke Software Engineer | Cloudera gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke