> 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
>>>> 
>>> 
>> 

Reply via email to