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

Compare with Current View Page History

« Previous Version 17 Next »

Status

Current state: Under Discussion

Discussion 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

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-minisr-partitions" to list out exactly which topic partitions are below "min.insync.replicas".

Public Interfaces

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

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 same technique `LogManager` uses to determine the "min.insync.replicas" for a given topic:

(1) Get broker/cluster-level configuration (from `AdminClient.describeConfigs`)

(2) Get topic-level configuration (from `KafkaZkClient.getLogConfigs` while passing in default "min.insync.replicas" as the value retrieved from (1))

We must pass the default value as a parameter in (2) as `KafkaZkClient.getLogConfigs` will return a default value if the specified configuration is not set in Zookeeper (Kafka default for "min.insync.replicas" is 1).

If the user has configured "min.insync.replicas" on the broker/cluster-level (and not topic-level), then `KafkaZkClient.getLogConfigs` will return the default value of 1 instead of the broker/cluster-level value, so we must make sure to pass in the value used by the broker/cluster as the "default" value.

This means we must add an additional flag "--bootstrap-server" to use AdminClient to get the broker/cluster "min.insync.replicas" configuration to use as the default when it is not found in Zookeeper.


Example of using the command:

bin/kafka-topics.sh --zookeeper localhost:2181 --describe --under-min-isr-partitions --bootstrap-server localhost:9092


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

None so far.

  • No labels