Table of Contents |
---|
Status
Current state: ACCEPTED
Discussion thread: DISCUSSIONhere
Discussion Vote thread: here
JIRA: KAFKA-7904
PR: https://github.com/apache/kafka/pull/6421
Motivation
Today, a topic partition may be categorized as:
...
UnderReplicated: A partition in which the isr_set_size is not equal to the replica_set_size (isr_set_size can be bigger or smaller than replica_replication_factor)
AtMinIsr: A partition in which the isr_set_size is equal to the min_isr_size, which also means a partition in which 1 more drop in isr_set_size will lead to at least producer (acks=ALL) failure
...
When an admin triggers a repartition, the ISR set is first expanded from [old_set] to [old_set + new_set], and then reduced to just the [new_set]. In this case, UnderReplicated will be non-zero even when the ISR set is [old_set + new_set]. AtMinIsr will not be non-zero during [old_set + new_set] step unless something goes wrong during repartitioning and replicas are failing to fetch (reducing the isr_set_size to min_isr_size), but we want to know if this happens.
(2) min.insync.replicas = 1 and replication-factor > 2
The default value for this configuration is 1, and users can change this to provide higher durability guarantees. In the default scenario where min.insync.replicas = 1 and replication-factor = 3, the AtMinIsr metric will be non-zero when isr_set_size = 1, which tells us that 1 more drop in this set will lead to a completely unavailable partition. This is very powerful for users that have min.insync.replicas = 1 and replication-factor > 2.
...