You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 22 Next »

Status

Current stateDISCUSSION

Discussion threadhere

JIRA: KAFKA-7904

Motivation

Today, a topic partition may be categorized as:

(1) Fully in sync (inSyncReplicas == allReplicasMap)

(2) UnderReplicated

(3) UnderMinIsr

(4) Offline


(3) and (4) are failure scenarios in which clients will face unavailability (producers with acks=ALL will fail to produce if ISR count is under the configured "min.insync.replicas" count).

(2) Under-replicated partitions occur whenever the inSyncReplicas set is not equal to the allReplicasMap, which can happen when we have:

  • Repartitioning
  • Broker restarts
  • Transient network issues
  • Broker failures


The current categorization of topic partitions has a gap as an UnderReplicatedPartition is generic and triggered in the various situations listed above. This makes it incredibly difficult to use UnderReplicatedPartitions as an indicator for alerting as alerts configured on this metric will trigger whenever there is a change in the ISR set.


In reality, we can actually tolerate a reduced ISR set as long as it meets the minimum insync replica count (as configured by the "min.insync.replicas" configuration), otherwise producers with acks=ALL configured will fail. 


This KIP aims to improve monitoring by proposing a new metric group AtMinIsr, which consists of partitions that only have the minimum number of insync replicas remaining in the ISR set (as configured by "min.insync.replicas"). This new metric can be used as a warning since any partitions that are AtMinIsr are at danger of causing producer unavailability (for acks=ALL producers) if one more replica drops out of the ISR set.

Examples

Example 1:

1 partition

minIsrCount=2

ISR set = [0,1,2]


1. Broker 0 fails

  • ISR set = [1,2]
  • partition is UnderReplicatedPartition and AtMinIsr

2. Broker 1 fails

  • ISR set = [2]
  • partition is UnderReplicatedPartition and UnderMinIsr

In this example, AtMinIsr triggers when UnderReplicatedPartition triggers and tells us that 1 more failure will cause producers with ack=ALL to be unavailable.


Example 2: 

1 partition

minIsrCount=1

ISR set = [0,1,2]


1. Broker 0 fails

  • ISR set = [1,2]
  • partition is UnderReplicatedPartition

2. Broker 1 fails

  • ISR set = [2]
  • partition is UnderReplicatedPartition and AtMinIsr

3. Broker 2 fails

  • ISR set = []
  • partition is OfflinePartition

In this example, AtMinIsr triggers when there is only 1 insync replica remaining, and tells us that 1 more failure will cause producers with ack=ALL to be unavailable (the whole partition will be unavailable in this scenario).

Usage

A potential usage of this new AtMinIsr category is:

  1. Set up an alert to trigger when AtMinIsr > 0 for a period of time
  2. If the alert is triggered, then assess the health of the cluster:
    1. If there is an ongoing maintenance, then no action is needed
    2. Otherwise a broker may be unhealthy. The AtMinIsr partition metric or --at-min-isr-partitions TopicCommand option can be used to determine the list of topics to repartition if the unhealthy broker(s) cannot be fixed quickly

AtMinIsr Values + Possible Explanations

1. AtMinIsr is zero

Everything is fine, and business as usual. Nothing to do here.

2. AtMinIsr is consistently greater than zero for a prolonged period of time

Broker(s) may have failed so this could warrant an alert for an operator to take a look at the health of the cluster to see if any brokers are downed.

3. AtMinIsr bounces between zero and non-zero repeatedly

Broker(s) may be experiencing trouble such as high load or temporary network issues which is causing the partition to temporarily fall out of sync.

NOTE: There are still scenarios in which AtMinIsr will be non-zero during planned maintenance. For example, if RF=3 and minIsr is set to 2, then a planned of a broker can cause AtMinIsr to be non-zero. This however should not be occurring outside of planned maintenance.

Public Interfaces

We will introduce two new metrics and a new TopicCommand option to identify AtMinIsr partitions.

# New metrics
# Broker metric
- kafka.server:name=AtMinIsrPartitionCount,type=ReplicaManager
# Partition metric
- kafka.cluster:name=AtMinIsr,type=Partition,topic={topic},partition={partition}

# New TopicCommand option
--at-min-isr-partitions

Proposed Changes

We will add the gauge to Partition.scala:

newGauge("AtMinIsr",
  new Gauge[Int] {
    def value = {
      if (isAtMinIsr) 1 else 0
    }
  },
  tags
)


...


def isAtMinIsr: Boolean = {
  leaderReplicaIfLocal match {
    case Some(leaderReplica) =>
      inSyncReplicas.size == leaderReplica.log.get.config.minInSyncReplicas
    case None =>
      false
  }
}

And the similar change to ReplicaManager.scala:

val atMinIsrPartitionCount = newGauge(
  "AtMinIsrPartitionCount",
  new Gauge[Int] {
    def value = leaderPartitionsIterator.count(_.isAtMinIsr)
  }
)

And TopicCommand.scala:

private val reportAtMinIsrPartitionsOpt = parser.accepts("at-min-isr-partitions",
  "if set when describing topics, only show partitions whose isr count is equal to the configured minimum. Not supported with the --zookeeper option.")

private def hasAtMinIsrPartitions(partitionDescription: PartitionDescription) = {
  partitionDescription.isr.size == partitionDescription.minIsrCount
}

private def shouldPrintAtMinIsrPartitions(partitionDescription: PartitionDescription) = {
  opts.reportAtMinIsrPartitions && hasAtMinIsrPartitions(partitionDescription)
}

# Some other minor changes omitted

Compatibility, Deprecation, and Migration Plan

The new TopicCommand option requires use of AdminClient so it will not be available with the --zookeeper option.

Rejected Alternatives

None so far.

  • No labels