[ 
https://issues.apache.org/jira/browse/KAFKA-5341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dong Lin updated KAFKA-5341:
----------------------------
    Description: 
We currently have under replicated partitions, but we do not have a metric to 
track the number of partitions whose in-sync replicas count < minIsr. 
Partitions whose in-syn replicas count < minIsr will be unavailable to those 
producers who uses ack = all. It is important for Kafka operators to be 
notified of the existence of such partition because their existence reduces the 
availability of the Kafka service.

More specifically, we can define a per-broker metric UnderMinIsrPartitionCount 
as "The number of partitions that this broker leads for which in-sync replicas 
count < minIsr." So if the RF was 3, and min ISR is 2, then when there are 2 
replicas in ISR this partition would be in the under replicated partitions 
count. When there is 1 replica in ISR, this partition would also be in the 
UnderMinIsrPartitionCount.

See 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-164-+Add+UnderMinIsrPartitionCount+and+per-partition+UnderMinIsr+metrics
 for more detail.

  was:We currently have under replicated partitions, but we do not have a 
metrics for unavailable partitions. Unavailable partitions could be most easily 
defined as “The number of partitions that this broker leads for which the ISR 
is insufficient to meet the minimum ISR required.” So if the RF was 3, and min 
ISR is 2, then when there are 2 replicas in ISR this partition would be in the 
under replicated partitions count. When there is 1 replica in ISR, this 
partition would also be in the unavailable partitions count.


> Add UnderMinIsrPartitionCount and per-partition UnderMinIsr metrics
> -------------------------------------------------------------------
>
>                 Key: KAFKA-5341
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5341
>             Project: Kafka
>          Issue Type: New Feature
>            Reporter: Dong Lin
>            Assignee: Dong Lin
>
> We currently have under replicated partitions, but we do not have a metric to 
> track the number of partitions whose in-sync replicas count < minIsr. 
> Partitions whose in-syn replicas count < minIsr will be unavailable to those 
> producers who uses ack = all. It is important for Kafka operators to be 
> notified of the existence of such partition because their existence reduces 
> the availability of the Kafka service.
> More specifically, we can define a per-broker metric 
> UnderMinIsrPartitionCount as "The number of partitions that this broker leads 
> for which in-sync replicas count < minIsr." So if the RF was 3, and min ISR 
> is 2, then when there are 2 replicas in ISR this partition would be in the 
> under replicated partitions count. When there is 1 replica in ISR, this 
> partition would also be in the UnderMinIsrPartitionCount.
> See 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-164-+Add+UnderMinIsrPartitionCount+and+per-partition+UnderMinIsr+metrics
>  for more detail.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to