Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-04-04 Thread James Cheng



> On Apr 2, 2020, at 4:27 AM, Paolo Moriello  wrote:
> 
> Hi,
> 
> Thanks a lot for your feedback, I really appreciate your help on this.
> 
> Given what you suggested, I will take some time to update the kip with a
> proposal to make invalid configuration requests FAIL. This involves
> checking multiple code paths, as James was saying, to eg.: validate topic
> creation, topic-configuration changes, partition reassignment and broker
> configuration setup.
> 
> Regarding the latter, do you have any suggestion on what's the best thing
> to do? For instance, we know that we can specify min.insync.replicas at
> cluster level. At the same time, we can also specify
> default.replication.factor. If there is an inconsistency with this setup,
> do we want to make kafka fail at startup or allow the users to overwrite it
> at a later point? (I believe we should be strict here and fail at startup).
> A similar question applies to offsets.topic.replication.factor.
> 

Paolo,

I haven't had a chance to think of it deeply, but your idea of having kafka 
fail at startup makes sense to me. I'd like one of the committers to chime in 
about that idea, too.

Some broker default settings can also be set dynamically during runtime, so you 
will also have to catch/reject those.

-James

> Thanks,
> Paolo
> 
> On Wed, 1 Apr 2020 at 05:29, James Cheng  wrote:
> 
>> I agree that we should prevent the creation of such a topic configuration.
>> That would mean catching it at topic-creation time, as well as catching it
>> on any topic-configuration changes that might make min.isr > replication
>> factor.
>> 
>> Not sure how we would detect things if someone changed the broker-default
>> configuration. That could be tricky.
>> 
>> Btw, I was the person who filed the original JIRA and as Mickael guessed,
>> it was done by mistake.
>> 
>> -James
>> 
>>> On Mar 31, 2020, at 9:30 AM, Ismael Juma  wrote:
>>> 
>>> Hi Paolo,
>>> 
>>> Thanks for the KIP. Why would one want to set min.isr to be higher than
>>> replication factor even in that case? Mickael's suggestion seems better
>> to
>>> me.
>>> 
>>> Ismael
>>> 
>>> On Fri, Mar 13, 2020 at 10:28 AM Paolo Moriello <
>> paolomoriell...@gmail.com>
>>> wrote:
>>> 
 Hi Mickael,
 
 Thanks for your interest in this. The main motivation to NOT make topic
 creation fail when this mismatch happens is because at the moment it is
 possible to produce/consume on topics if acks is not set to all. I'm not
 sure we want to disable this behavior (as we would by failing at topic
 creation). That's why I decided to go for a softer approach, which at
>> least
 gives some more clarity to the users and avoids other issues mentioned
>> in
 the KIP.
 
 Let's see what others think!
 
 On Fri, 13 Mar 2020 at 17:16, Mickael Maison 
 wrote:
 
> Hi Paolo,
> 
> Thanks for looking at this issue. This can indeed be a source of
 confusion.
> 
> I'm wondering if we should prevent the creation of topics with
> min.insync.replicas > replication.factor?
> You listed that as a rejected alternative because it requires more
> changes. However, I can't think of any scenarios where a user would
> want to create such a topic. I'm guessing it's probably always by
> mistake.
> 
> Let's see what other people think but I think it's worth checking what
> needs to be done if we wanted to prevent topics with bogus configs
> 
> On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
>  wrote:
>> 
>> Hi,
>> 
>> Following this Jira ticket (
> https://issues.apache.org/jira/browse/KAFKA-4680),
>> I've created a proposal (
>> 
> 
 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor
> )
>> to add a new exception/error to be used on min.insync.replicas >
>> replication.factor.
>> 
>> The proposal aims to introduce a new exception specific for the
>> configuration mismatch above to be used when producers requires acks =
> all.
>> At the moment we are using NotEnoughReplicaException, which is a
> retriable
>> exception and is used to fail on insync replicas < min isr. Plan is to
> have
>> a new, non-retriable exception, to separate the two cases.
>> 
>> I've also submitted a PR for the change mentioned above:
>> https://github.com/apache/kafka/pull/8225
>> 
>> Please have a look and let me know what you think.
>> 
>> Thanks,
>> Paolo
> 
 
>> 
>> 



Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-04-02 Thread Paolo Moriello
Hi,

Thanks a lot for your feedback, I really appreciate your help on this.

Given what you suggested, I will take some time to update the kip with a
proposal to make invalid configuration requests FAIL. This involves
checking multiple code paths, as James was saying, to eg.: validate topic
creation, topic-configuration changes, partition reassignment and broker
configuration setup.

Regarding the latter, do you have any suggestion on what's the best thing
to do? For instance, we know that we can specify min.insync.replicas at
cluster level. At the same time, we can also specify
default.replication.factor. If there is an inconsistency with this setup,
do we want to make kafka fail at startup or allow the users to overwrite it
at a later point? (I believe we should be strict here and fail at startup).
A similar question applies to offsets.topic.replication.factor.

Thanks,
Paolo

On Wed, 1 Apr 2020 at 05:29, James Cheng  wrote:

> I agree that we should prevent the creation of such a topic configuration.
> That would mean catching it at topic-creation time, as well as catching it
> on any topic-configuration changes that might make min.isr > replication
> factor.
>
> Not sure how we would detect things if someone changed the broker-default
> configuration. That could be tricky.
>
> Btw, I was the person who filed the original JIRA and as Mickael guessed,
> it was done by mistake.
>
> -James
>
> > On Mar 31, 2020, at 9:30 AM, Ismael Juma  wrote:
> >
> > Hi Paolo,
> >
> > Thanks for the KIP. Why would one want to set min.isr to be higher than
> > replication factor even in that case? Mickael's suggestion seems better
> to
> > me.
> >
> > Ismael
> >
> > On Fri, Mar 13, 2020 at 10:28 AM Paolo Moriello <
> paolomoriell...@gmail.com>
> > wrote:
> >
> >> Hi Mickael,
> >>
> >> Thanks for your interest in this. The main motivation to NOT make topic
> >> creation fail when this mismatch happens is because at the moment it is
> >> possible to produce/consume on topics if acks is not set to all. I'm not
> >> sure we want to disable this behavior (as we would by failing at topic
> >> creation). That's why I decided to go for a softer approach, which at
> least
> >> gives some more clarity to the users and avoids other issues mentioned
> in
> >> the KIP.
> >>
> >> Let's see what others think!
> >>
> >> On Fri, 13 Mar 2020 at 17:16, Mickael Maison 
> >> wrote:
> >>
> >>> Hi Paolo,
> >>>
> >>> Thanks for looking at this issue. This can indeed be a source of
> >> confusion.
> >>>
> >>> I'm wondering if we should prevent the creation of topics with
> >>> min.insync.replicas > replication.factor?
> >>> You listed that as a rejected alternative because it requires more
> >>> changes. However, I can't think of any scenarios where a user would
> >>> want to create such a topic. I'm guessing it's probably always by
> >>> mistake.
> >>>
> >>> Let's see what other people think but I think it's worth checking what
> >>> needs to be done if we wanted to prevent topics with bogus configs
> >>>
> >>> On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
> >>>  wrote:
> 
>  Hi,
> 
>  Following this Jira ticket (
> >>> https://issues.apache.org/jira/browse/KAFKA-4680),
>  I've created a proposal (
> 
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor
> >>> )
>  to add a new exception/error to be used on min.insync.replicas >
>  replication.factor.
> 
>  The proposal aims to introduce a new exception specific for the
>  configuration mismatch above to be used when producers requires acks =
> >>> all.
>  At the moment we are using NotEnoughReplicaException, which is a
> >>> retriable
>  exception and is used to fail on insync replicas < min isr. Plan is to
> >>> have
>  a new, non-retriable exception, to separate the two cases.
> 
>  I've also submitted a PR for the change mentioned above:
>  https://github.com/apache/kafka/pull/8225
> 
>  Please have a look and let me know what you think.
> 
>  Thanks,
>  Paolo
> >>>
> >>
>
>


Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-03-31 Thread James Cheng
I agree that we should prevent the creation of such a topic configuration. That 
would mean catching it at topic-creation time, as well as catching it on any 
topic-configuration changes that might make min.isr > replication factor.

Not sure how we would detect things if someone changed the broker-default 
configuration. That could be tricky.

Btw, I was the person who filed the original JIRA and as Mickael guessed, it 
was done by mistake.

-James

> On Mar 31, 2020, at 9:30 AM, Ismael Juma  wrote:
> 
> Hi Paolo,
> 
> Thanks for the KIP. Why would one want to set min.isr to be higher than
> replication factor even in that case? Mickael's suggestion seems better to
> me.
> 
> Ismael
> 
> On Fri, Mar 13, 2020 at 10:28 AM Paolo Moriello 
> wrote:
> 
>> Hi Mickael,
>> 
>> Thanks for your interest in this. The main motivation to NOT make topic
>> creation fail when this mismatch happens is because at the moment it is
>> possible to produce/consume on topics if acks is not set to all. I'm not
>> sure we want to disable this behavior (as we would by failing at topic
>> creation). That's why I decided to go for a softer approach, which at least
>> gives some more clarity to the users and avoids other issues mentioned in
>> the KIP.
>> 
>> Let's see what others think!
>> 
>> On Fri, 13 Mar 2020 at 17:16, Mickael Maison 
>> wrote:
>> 
>>> Hi Paolo,
>>> 
>>> Thanks for looking at this issue. This can indeed be a source of
>> confusion.
>>> 
>>> I'm wondering if we should prevent the creation of topics with
>>> min.insync.replicas > replication.factor?
>>> You listed that as a rejected alternative because it requires more
>>> changes. However, I can't think of any scenarios where a user would
>>> want to create such a topic. I'm guessing it's probably always by
>>> mistake.
>>> 
>>> Let's see what other people think but I think it's worth checking what
>>> needs to be done if we wanted to prevent topics with bogus configs
>>> 
>>> On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
>>>  wrote:
 
 Hi,
 
 Following this Jira ticket (
>>> https://issues.apache.org/jira/browse/KAFKA-4680),
 I've created a proposal (
 
>>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor
>>> )
 to add a new exception/error to be used on min.insync.replicas >
 replication.factor.
 
 The proposal aims to introduce a new exception specific for the
 configuration mismatch above to be used when producers requires acks =
>>> all.
 At the moment we are using NotEnoughReplicaException, which is a
>>> retriable
 exception and is used to fail on insync replicas < min isr. Plan is to
>>> have
 a new, non-retriable exception, to separate the two cases.
 
 I've also submitted a PR for the change mentioned above:
 https://github.com/apache/kafka/pull/8225
 
 Please have a look and let me know what you think.
 
 Thanks,
 Paolo
>>> 
>> 



Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-03-31 Thread Ismael Juma
Hi Paolo,

Thanks for the KIP. Why would one want to set min.isr to be higher than
replication factor even in that case? Mickael's suggestion seems better to
me.

Ismael

On Fri, Mar 13, 2020 at 10:28 AM Paolo Moriello 
wrote:

> Hi Mickael,
>
> Thanks for your interest in this. The main motivation to NOT make topic
> creation fail when this mismatch happens is because at the moment it is
> possible to produce/consume on topics if acks is not set to all. I'm not
> sure we want to disable this behavior (as we would by failing at topic
> creation). That's why I decided to go for a softer approach, which at least
> gives some more clarity to the users and avoids other issues mentioned in
> the KIP.
>
> Let's see what others think!
>
> On Fri, 13 Mar 2020 at 17:16, Mickael Maison 
> wrote:
>
> > Hi Paolo,
> >
> > Thanks for looking at this issue. This can indeed be a source of
> confusion.
> >
> > I'm wondering if we should prevent the creation of topics with
> > min.insync.replicas > replication.factor?
> > You listed that as a rejected alternative because it requires more
> > changes. However, I can't think of any scenarios where a user would
> > want to create such a topic. I'm guessing it's probably always by
> > mistake.
> >
> > Let's see what other people think but I think it's worth checking what
> > needs to be done if we wanted to prevent topics with bogus configs
> >
> > On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
> >  wrote:
> > >
> > > Hi,
> > >
> > > Following this Jira ticket (
> > https://issues.apache.org/jira/browse/KAFKA-4680),
> > > I've created a proposal (
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor
> > )
> > > to add a new exception/error to be used on min.insync.replicas >
> > > replication.factor.
> > >
> > > The proposal aims to introduce a new exception specific for the
> > > configuration mismatch above to be used when producers requires acks =
> > all.
> > > At the moment we are using NotEnoughReplicaException, which is a
> > retriable
> > > exception and is used to fail on insync replicas < min isr. Plan is to
> > have
> > > a new, non-retriable exception, to separate the two cases.
> > >
> > > I've also submitted a PR for the change mentioned above:
> > > https://github.com/apache/kafka/pull/8225
> > >
> > > Please have a look and let me know what you think.
> > >
> > > Thanks,
> > > Paolo
> >
>


Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-03-30 Thread Paolo Moriello
Hi MAnna,

Thanks for your feedback. I think you are right,
INCONSISTENT_REPLICATION_FACTOR sounds more specific and closer to the
point of this kip. Will update the KIP and the PR accordingly.

Thanks,
Paolo

On Mon, 30 Mar 2020 at 13:06, M. Manna  wrote:

> Hey Paolo,
>
> Overall LGTM. I only have one suggestion.
>
> You are planning to call it as "INCONSISTENT_REPLICA_CONFIGURATION".
>
> How about if we call it "INCONSISTENT_REPLICATION_FACTOR"? Replica
> configuration might also mean that one of the configuration is not
> consistent. But with "INCONSISTENT_REPLICATION_FACTOR" it's semantically
> closer to your KIP motivation. Also, users will likely and inherently,
> discover that it's ISR and replication factor that might've been wrong for
> them.
>
> Perhaps, someone else may have different ideas, but please feel free to
> consider it.
>
> Regards,
> M. MAnna
>
> On Mon, 30 Mar 2020 at 12:55, Paolo Moriello 
> wrote:
>
> > Hi,
> >
> > Any other feedback on this before we start the vote?
> >
> > Paolo
> >
> > On Fri, 13 Mar 2020 at 17:28, Paolo Moriello 
> > wrote:
> >
> > > Hi Mickael,
> > >
> > > Thanks for your interest in this. The main motivation to NOT make topic
> > > creation fail when this mismatch happens is because at the moment it is
> > > possible to produce/consume on topics if acks is not set to all. I'm
> not
> > > sure we want to disable this behavior (as we would by failing at topic
> > > creation). That's why I decided to go for a softer approach, which at
> > least
> > > gives some more clarity to the users and avoids other issues mentioned
> in
> > > the KIP.
> > >
> > > Let's see what others think!
> > >
> > > On Fri, 13 Mar 2020 at 17:16, Mickael Maison  >
> > > wrote:
> > >
> > >> Hi Paolo,
> > >>
> > >> Thanks for looking at this issue. This can indeed be a source of
> > >> confusion.
> > >>
> > >> I'm wondering if we should prevent the creation of topics with
> > >> min.insync.replicas > replication.factor?
> > >> You listed that as a rejected alternative because it requires more
> > >> changes. However, I can't think of any scenarios where a user would
> > >> want to create such a topic. I'm guessing it's probably always by
> > >> mistake.
> > >>
> > >> Let's see what other people think but I think it's worth checking what
> > >> needs to be done if we wanted to prevent topics with bogus configs
> > >>
> > >> On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
> > >>  wrote:
> > >> >
> > >> > Hi,
> > >> >
> > >> > Following this Jira ticket (
> > >> https://issues.apache.org/jira/browse/KAFKA-4680),
> > >> > I've created a proposal (
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor
> > >> )
> > >> > to add a new exception/error to be used on min.insync.replicas >
> > >> > replication.factor.
> > >> >
> > >> > The proposal aims to introduce a new exception specific for the
> > >> > configuration mismatch above to be used when producers requires
> acks =
> > >> all.
> > >> > At the moment we are using NotEnoughReplicaException, which is a
> > >> retriable
> > >> > exception and is used to fail on insync replicas < min isr. Plan is
> to
> > >> have
> > >> > a new, non-retriable exception, to separate the two cases.
> > >> >
> > >> > I've also submitted a PR for the change mentioned above:
> > >> > https://github.com/apache/kafka/pull/8225
> > >> >
> > >> > Please have a look and let me know what you think.
> > >> >
> > >> > Thanks,
> > >> > Paolo
> > >>
> > >
> >
>


Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-03-30 Thread M. Manna
Hey Paolo,

Overall LGTM. I only have one suggestion.

You are planning to call it as "INCONSISTENT_REPLICA_CONFIGURATION".

How about if we call it "INCONSISTENT_REPLICATION_FACTOR"? Replica
configuration might also mean that one of the configuration is not
consistent. But with "INCONSISTENT_REPLICATION_FACTOR" it's semantically
closer to your KIP motivation. Also, users will likely and inherently,
discover that it's ISR and replication factor that might've been wrong for
them.

Perhaps, someone else may have different ideas, but please feel free to
consider it.

Regards,
M. MAnna

On Mon, 30 Mar 2020 at 12:55, Paolo Moriello 
wrote:

> Hi,
>
> Any other feedback on this before we start the vote?
>
> Paolo
>
> On Fri, 13 Mar 2020 at 17:28, Paolo Moriello 
> wrote:
>
> > Hi Mickael,
> >
> > Thanks for your interest in this. The main motivation to NOT make topic
> > creation fail when this mismatch happens is because at the moment it is
> > possible to produce/consume on topics if acks is not set to all. I'm not
> > sure we want to disable this behavior (as we would by failing at topic
> > creation). That's why I decided to go for a softer approach, which at
> least
> > gives some more clarity to the users and avoids other issues mentioned in
> > the KIP.
> >
> > Let's see what others think!
> >
> > On Fri, 13 Mar 2020 at 17:16, Mickael Maison 
> > wrote:
> >
> >> Hi Paolo,
> >>
> >> Thanks for looking at this issue. This can indeed be a source of
> >> confusion.
> >>
> >> I'm wondering if we should prevent the creation of topics with
> >> min.insync.replicas > replication.factor?
> >> You listed that as a rejected alternative because it requires more
> >> changes. However, I can't think of any scenarios where a user would
> >> want to create such a topic. I'm guessing it's probably always by
> >> mistake.
> >>
> >> Let's see what other people think but I think it's worth checking what
> >> needs to be done if we wanted to prevent topics with bogus configs
> >>
> >> On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
> >>  wrote:
> >> >
> >> > Hi,
> >> >
> >> > Following this Jira ticket (
> >> https://issues.apache.org/jira/browse/KAFKA-4680),
> >> > I've created a proposal (
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor
> >> )
> >> > to add a new exception/error to be used on min.insync.replicas >
> >> > replication.factor.
> >> >
> >> > The proposal aims to introduce a new exception specific for the
> >> > configuration mismatch above to be used when producers requires acks =
> >> all.
> >> > At the moment we are using NotEnoughReplicaException, which is a
> >> retriable
> >> > exception and is used to fail on insync replicas < min isr. Plan is to
> >> have
> >> > a new, non-retriable exception, to separate the two cases.
> >> >
> >> > I've also submitted a PR for the change mentioned above:
> >> > https://github.com/apache/kafka/pull/8225
> >> >
> >> > Please have a look and let me know what you think.
> >> >
> >> > Thanks,
> >> > Paolo
> >>
> >
>


Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-03-30 Thread Paolo Moriello
Hi,

Any other feedback on this before we start the vote?

Paolo

On Fri, 13 Mar 2020 at 17:28, Paolo Moriello 
wrote:

> Hi Mickael,
>
> Thanks for your interest in this. The main motivation to NOT make topic
> creation fail when this mismatch happens is because at the moment it is
> possible to produce/consume on topics if acks is not set to all. I'm not
> sure we want to disable this behavior (as we would by failing at topic
> creation). That's why I decided to go for a softer approach, which at least
> gives some more clarity to the users and avoids other issues mentioned in
> the KIP.
>
> Let's see what others think!
>
> On Fri, 13 Mar 2020 at 17:16, Mickael Maison 
> wrote:
>
>> Hi Paolo,
>>
>> Thanks for looking at this issue. This can indeed be a source of
>> confusion.
>>
>> I'm wondering if we should prevent the creation of topics with
>> min.insync.replicas > replication.factor?
>> You listed that as a rejected alternative because it requires more
>> changes. However, I can't think of any scenarios where a user would
>> want to create such a topic. I'm guessing it's probably always by
>> mistake.
>>
>> Let's see what other people think but I think it's worth checking what
>> needs to be done if we wanted to prevent topics with bogus configs
>>
>> On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
>>  wrote:
>> >
>> > Hi,
>> >
>> > Following this Jira ticket (
>> https://issues.apache.org/jira/browse/KAFKA-4680),
>> > I've created a proposal (
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor
>> )
>> > to add a new exception/error to be used on min.insync.replicas >
>> > replication.factor.
>> >
>> > The proposal aims to introduce a new exception specific for the
>> > configuration mismatch above to be used when producers requires acks =
>> all.
>> > At the moment we are using NotEnoughReplicaException, which is a
>> retriable
>> > exception and is used to fail on insync replicas < min isr. Plan is to
>> have
>> > a new, non-retriable exception, to separate the two cases.
>> >
>> > I've also submitted a PR for the change mentioned above:
>> > https://github.com/apache/kafka/pull/8225
>> >
>> > Please have a look and let me know what you think.
>> >
>> > Thanks,
>> > Paolo
>>
>


Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-03-13 Thread Paolo Moriello
Hi Mickael,

Thanks for your interest in this. The main motivation to NOT make topic
creation fail when this mismatch happens is because at the moment it is
possible to produce/consume on topics if acks is not set to all. I'm not
sure we want to disable this behavior (as we would by failing at topic
creation). That's why I decided to go for a softer approach, which at least
gives some more clarity to the users and avoids other issues mentioned in
the KIP.

Let's see what others think!

On Fri, 13 Mar 2020 at 17:16, Mickael Maison 
wrote:

> Hi Paolo,
>
> Thanks for looking at this issue. This can indeed be a source of confusion.
>
> I'm wondering if we should prevent the creation of topics with
> min.insync.replicas > replication.factor?
> You listed that as a rejected alternative because it requires more
> changes. However, I can't think of any scenarios where a user would
> want to create such a topic. I'm guessing it's probably always by
> mistake.
>
> Let's see what other people think but I think it's worth checking what
> needs to be done if we wanted to prevent topics with bogus configs
>
> On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
>  wrote:
> >
> > Hi,
> >
> > Following this Jira ticket (
> https://issues.apache.org/jira/browse/KAFKA-4680),
> > I've created a proposal (
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor
> )
> > to add a new exception/error to be used on min.insync.replicas >
> > replication.factor.
> >
> > The proposal aims to introduce a new exception specific for the
> > configuration mismatch above to be used when producers requires acks =
> all.
> > At the moment we are using NotEnoughReplicaException, which is a
> retriable
> > exception and is used to fail on insync replicas < min isr. Plan is to
> have
> > a new, non-retriable exception, to separate the two cases.
> >
> > I've also submitted a PR for the change mentioned above:
> > https://github.com/apache/kafka/pull/8225
> >
> > Please have a look and let me know what you think.
> >
> > Thanks,
> > Paolo
>


Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-03-13 Thread Mickael Maison
Hi Paolo,

Thanks for looking at this issue. This can indeed be a source of confusion.

I'm wondering if we should prevent the creation of topics with
min.insync.replicas > replication.factor?
You listed that as a rejected alternative because it requires more
changes. However, I can't think of any scenarios where a user would
want to create such a topic. I'm guessing it's probably always by
mistake.

Let's see what other people think but I think it's worth checking what
needs to be done if we wanted to prevent topics with bogus configs

On Fri, Mar 13, 2020 at 3:28 PM Paolo Moriello
 wrote:
>
> Hi,
>
> Following this Jira ticket (https://issues.apache.org/jira/browse/KAFKA-4680),
> I've created a proposal (
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-579%3A+new+exception+on+min.insync.replicas+%3E+replication.factor)
> to add a new exception/error to be used on min.insync.replicas >
> replication.factor.
>
> The proposal aims to introduce a new exception specific for the
> configuration mismatch above to be used when producers requires acks = all.
> At the moment we are using NotEnoughReplicaException, which is a retriable
> exception and is used to fail on insync replicas < min isr. Plan is to have
> a new, non-retriable exception, to separate the two cases.
>
> I've also submitted a PR for the change mentioned above:
> https://github.com/apache/kafka/pull/8225
>
> Please have a look and let me know what you think.
>
> Thanks,
> Paolo