Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current state: ACCEPTED

Discussion thread: DISCUSSIONhere

Discussion Vote threadhere

JIRA: KAFKA-7904

PRhttps://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

...

(1) Repartitioning

When an admin triggers a repartitioningrepartition, 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.

...

One example is if we have a cluster with massive load and we do not want to trigger a repartition to make isr_set_size = replica_replication_factor unless absolutely necessary as repartitioning introduces additional load which can impact clients. Maybe we also expect the failed broker to be restored soon so we don't want to do anything unless absolutely necessary. In these scenarios, the AtMinIsr metric will tell us when we absolutely need to consider repartitioning or some other action to restore the health of the cluster (false negative is still possible but it tells us that we could not tolerate any more failure at the time it was non-zero if we do not want acks=ALL producers to fail).In our Kafka environment, we do not even have alerts configured for UnderReplicated as it is too noisy for us and we can tolerate some failures. We run a periodic job to perform the same functionality as AtMinIsr, but it would be better to have it as a metric so we can configure an alert on it.

Usage

A potential usage of this new AtMinIsr category is:

...