Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-24 Thread James Cheng
I read the recent Client Survey (https://www.confluent.io/blog/first-annual-state-apache-kafka-client-use-survey/ ). It said that most responders to the survey said that reliability was critical or very

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-03 Thread Grant Henke
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

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-03 Thread Ismael Juma
I'd be in favour too. Ismael On 3 Feb 2017 7:33 am, "Ewen Cheslack-Postava" wrote: > On Thu, Feb 2, 2017 at 11:21 PM, James Cheng wrote: > > > Ewen, > > > > Ah right, that's a good point. > > > > My initial reaction to your examples was that "well,

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-02 Thread Ewen Cheslack-Postava
On Thu, Feb 2, 2017 at 11:21 PM, James Cheng 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.

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-02 Thread James Cheng
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

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-02 Thread Ewen Cheslack-Postava
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)

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-01-27 Thread James Cheng
> On Jan 27, 2017, at 12:18 AM, Ewen Cheslack-Postava wrote: > > On Thu, Jan 26, 2017 at 4:23 PM, Luciano Afranllie > wrote: > >> I was thinking about the situation where you have less brokers in the ISR >> list than the number set in

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-01-27 Thread Ewen Cheslack-Postava
On Thu, Jan 26, 2017 at 4:23 PM, Luciano Afranllie 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

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-01-26 Thread Luciano Afranllie
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

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-01-26 Thread Ewen Cheslack-Postava
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

Trying to understand design decision about producer ack and min.insync.replicas

2017-01-24 Thread Luciano Afranllie
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