Status
Current state: Accepted
Discussion thread:
JIRA: KAFKA-5341
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
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.
Public Interfaces
Add the following two yammer metrics (and resulting JMX metrics)
- kafka.server:name=UnderMinIsrPartitionCount,type=ReplicaManager
There is one such gauge per-broker
- kafka.cluster:name=UnderMinIsr,type=Partition,topic={topic},partition={partition}
There is one such gauge per-partition.
Proposed Changes
- Add the yammer gauge UnderMinIsr
to the kafka.cluster.Partition
class, similar to the existing UnderReplicated
metric.
The value of this metric is 1 if the broker is leader of this partition AND the number of in-sync replicas of this partition < minIsr of this partition. Otherwise it is 0.
- Add the yammer gauge UnderMInIsrPartitionCount
to the kafka.server.ReplicaManager
class, similar to the existing UnderReplicatedPartitions
metric.
The value of this metric is the total number of leader partitions on this broker whose in-sync replicas < minIsr (as defined above).
Compatibility, Deprecation, and Migration Plan
The change is fully backwards compatible.
Rejected Alternatives
None