Hey Kevin,

Thanks for the proposal!

It seems that the proposed implementation does not match the motivation.
The motivation suggests that the operator wants to tell the planned
maintenance (e.g. broker restart) from unplanned failure (e.g. network
failure). But the use of the metric AtMinIsr does not really differentiate
between these causes of the reduced number of ISR. For example, an
unplanned failure can cause ISR to drop from 3 to 2 but it can still be
higher than the minIsr (say 1). And a planned maintenance can cause ISR to
drop from 3 to 2, which trigger the AtMinIsr metric if minIsr=2. Can you
update the design doc to fix or explain this issue?

Thanks,
Dong

On Tue, Feb 12, 2019 at 9:02 AM Kevin Lu <lu.ke...@berkeley.edu> wrote:

> Hi All,
>
> Getting the discussion thread started for KIP-427 in case anyone is free
> right now.
>
> I’d like to propose a new category of topic partitions *AtMinIsr* which are
> partitions that only have the minimum number of in sync replicas left in
> the ISR set (as configured by min.insync.replicas).
>
> This would add two new metrics *ReplicaManager.AtMinIsrPartitionCount *&
> *Partition.AtMinIsr*, and a new TopicCommand option*
> --at-min-isr-partitions* to help in monitoring and alerting.
>
> KIP link: KIP-427: Add AtMinIsr topic partition category (new metric &
> TopicCommand option)
> <
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=103089398
> >
>
> Please take a look and let me know what you think.
>
> Regards,
> Kevin
>

Reply via email to