Hey Kevin, Thanks for the KIP. I like Mickael's suggestion to add --under-minisr-partitions since it fits with the metric we already expose. It's also a good question whether there should be a separate category for partitions which are right at min.isr. I'm reluctant to add new categories, but I agree there might be a gap at the moment. Say you have a replication factor of 3 and the min isr is set to 1. Our notion of URP does not capture the difference between having an ISR down to a size of 1 and one down to a size of 2. The reason this might be significant is that a shrink of the ISR down to 2 may just be caused by a rolling restart or a transient network blip. A shrink to 1, on the other hand, might be indicative of a more severe problem and could be cause for a call from pagerduty.
-Jason On Thu, Aug 2, 2018 at 9:28 AM, Kevin Lu <lu.ke...@berkeley.edu> wrote: > Hi Mickael, > > Thanks for the suggestion! > > Correct me if I am mistaken, but if a producer attempts to send to a > partition that is under min ISR (and ack=all or -1) then the send will fail > with a NotEnoughReplicas or NotEnoughReplicasAfterAppend exception? At this > point, client-side has already suffered failure but the server-side is > still fine for now? > > If the above is true, then this would be a FATAL case for producers. > > Would it be valuable to include the CRITICAL case where a topic partition > has exactly min ISR so that Kafka operators can take action so it does not > become FATAL? This could be in the same option or a new one. > > Thanks! > > Regards, > Kevin > > On Thu, Aug 2, 2018 at 2:27 AM Mickael Maison <mickael.mai...@gmail.com> > wrote: > > > What about also adding a --under-minisr-partitions option? > > > > That would match the > > "kafka.server:type=ReplicaManager,name=UnderMinIsrPartitionCount" > > broker metric and it's usually pretty relevant when investigating > > issues > > > > On Thu, Aug 2, 2018 at 8:54 AM, Kevin Lu <lu.ke...@berkeley.edu> wrote: > > > Hi friends! > > > > > > This thread is to discuss KIP-351 > > > < > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-351%3A > +Add+--critical-partitions+option+to+describe+topics+command > > > > > > ! > > > > > > I am proposing to add a --critical-partitions option to the describe > > topics > > > command that will only list out topic partitions that have 1 ISR left > > (RF > > > > 1) as they would be in a critical state and need immediate > > repartitioning. > > > > > > I wonder if the name "critical" is appropriate? > > > > > > Thoughts? > > > > > > Thanks! > > > > > > Regards, > > > Kevin > > >