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 <manme...@gmail.com> 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 <paolomoriell...@gmail.com> > wrote: > > > Hi, > > > > Any other feedback on this before we start the vote? > > > > Paolo > > > > On Fri, 13 Mar 2020 at 17:28, 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 <mickael.mai...@gmail.com > > > > > 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 > > >> <paolomoriell...@gmail.com> 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 > > >> > > > > > >