Versions Compared

Key

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

Table of Contents

Status

Current state: Under ACCEPTED

Discussion thread: here

Previous Discussion thread: here

Vote thread: here

JIRA: KAFKA-7236

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

A topic partition can be in one of four states (assuming replication factor of 3):

(ISR = in sync replica)

3/3 ISRs: OK

2/3 ISRs: WARNING (under-replicated partition)

1/3 ISRs: CRITICAL (under-replicated partition)

0/3 ISRs: FATAL (offline/unavailable partition)

TopicCommand already has the --under-replicated-partitions and --unavailable-partitions flags, but it would be beneficial to include an additional --critical-partitions option that specifically lists out partitions in CRITICAL state (only one remaining ISR left).

With this new option, Kafka users can use this option to identify the exact topic partitions that are critical and need immediate repartitioning. Kafka users can also set up critical alerts to trigger when the output of this command contains partitions.

A couple cases where identifying this CRITICAL state is useful in alerting:

  • Users that have a large amount of topics in a single cluster, making it incredibly hard to manually repartition all topics that have under-replicated partitions, so they only take action when it hits CRITICAL state
  • Users with a high replication-factor that can tolerate some broker failures and only take action when it hits CRITICAL state

...

PRhttps://github.com/apache/kafka/pull/6224

Motivation

The "min.insync.replicas" configuration specifies the minimum number of insync replicas required for a partition to accept messages from the producer. If the insync replica count of a partition falls under the specified "min.insync.replicas", then the broker will reject messages for producers using acks=all. These producers will suffer unavailability as they will see a NotEnoughReplicas or NotEnoughReplicasAfterAppend exception.

We currently have an UnderMinIsrPartitionCount metric which is useful for identifying when partitions fall under "min.insync.replicas", however it is still difficult to identify which topic partitions are affected and need fixing.

We can leverage the describe topics command in TopicCommand to add an option "--under-min-isr-partitions" to list out exactly which topic partitions are below "min.insync.replicas" and need fixing to maintain availability.

Public Interfaces

This change would

...

add an additional flag "--under-min-isr-partitions" to TopicCommand, but the output will follow the same format as the "under-replicated-partitions" and "offline-partitions" options.

Proposed Changes

When a user has specified the --critical-partitions option, TopicCommand will only print out topic partitions with ISR count equal to 1 if the replication factor of the topic is greater than 1.

We will not include topic partitions with a replication factor of 1 as they are intended to be single replica partitions so it would not be useful to list them out in this command.

The output will be in the same exact format as the --under-replicated-partitions and --unavailable-partitions options.

Compatibility, Deprecation, and Migration Plan

As this change adds a new option instead of modifying existing ones, there will not be any compatibility issues or a migration.

Rejected Alternatives

single-replica-partitions option

We could add this option that lists out all topic partitions that have only one in sync replica. This would include all partitions with a single in sync replica (RF >= 1).

...

Code Block
val reportUnderMinIsrPartitionsOpt = parser.accepts("under-min-isr-partitions",
  									     "if set when describing topics, only show partitions which are under the configured minimum in-sync replica count")


Proposed Changes

The challenge with supporting this additional feature is that the "min.insync.replicas" configuration may be set at a broker or topic level.

We can use the `AdminClient.describeConfigs` on the topics as that the API call will give us the "computed" proper values for configurations (ConfigSource as "DYNAMIC_TOPIC_CONFIG", "DYNAMIC_BROKER_CONFIG", "DYNAMIC_DEFAULT_BROKER_CONFIG", "STATIC_BROKER_CONFIG", and "DEFAULT_CONFIG").

We can pre-fetch the "computed" topic configurations if "--under-min-isr-partitions" option is specified to avoid making a separate AdminClient call per topic.

Code Block
# Assuming we have an AdminClient instance
val adminClient = ...

// Pre-fetch and get "computed" topic configs for all specified topics
val computedTopicConfigs = if (reportUnderMinIsrPartitions)
  Option(adminClient.describeConfigs(
    topics.map(topic => new ConfigResource(ConfigResource.Type.TOPIC, topic)).asJavaCollection).all().get()) else None

for (topic <- topics)
  ...
  if (describePartitions) {
    // Get "computed" topic "min.insync.replicas" for this topic
    val computedTopicMinISR = if (reportUnderMinISRPartitions)
      Option(computedTopicConfigs.get.get(new ConfigResource(ConfigResource.Type.TOPIC, topic))
      .get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value().toInt) else None

    for ((partitionId, assignedReplicas) <- sortedPartitions) {
      ...

      // Print current topic partition if reportUnderMinISRPartitions and ISR count < "computed" min ISR
      if (... ||
        (reportUnderMinISRPartitions && inSyncReplicas.size < computedTopicMinISR.get) {
      ...


This means we need an additional flag "--bootstrap-server" to use AdminClient. KIP-377: TopicCommand to use AdminClient is already proposing a change to use AdminClient and introduce a "--bootstrap-server" option, so we can leverage the changes in KIP-377 for this KIP.

NOTE: This option is not supported with the deprecated "--zookeeper" option.

Compatibility, Deprecation, and Migration Plan

Since we rely on AdminClient for the computed "min.insync.replicas" configuration, this new option CANNOT be used with the deprecated "--zookeeper" option.

Rejected Alternatives

None so far.