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 get the configured "min.insync.replicas" for a topic by:
(1) Check topic-level configuration in Zookeeper
(2) Use AdminClient to get broker/cluster-level configuration
This means we must add an additional flag "--bootstrap-server" to use AdminClient to describe broker configurations when we cannot find the configuration override in Zookeeper.
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.