Kafka UncleanLeaderElectionsPerSec > 0: confirmed silent data loss
Your UncleanLeaderElectionsPerSec alert fired. The JMX metric kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec shows OneMinuteRate > 0, or Count has incremented since your last check. This is confirmed data loss: a partition leader was elected from outside the ISR, and acknowledged records that the new leader does not possess are silently truncated.
Producers that received acks for those records were given a durability guarantee the cluster just broke. Stop additional loss, understand scope, and fix the conditions that allowed it.
With unclean.leader.election.enable=false, the default since Kafka 0.11.0.0, partitions whose entire ISR is unavailable go offline rather than sacrifice data. Setting it to true chooses availability over durability.
What this means
Kafka elects a new leader when the current leader is lost. The new leader must be an ISR member. If no ISR member is available and unclean.leader.election.enable is true, the controller can elect any replica, even one that is behind. That replica becomes the authoritative source of truth, and the previous leader’s divergent log suffix is discarded. Any record already acked to a producer in that suffix is gone without error.
The MBean exposes Count, a lifetime cumulative counter, and OneMinuteRate. Because Count never resets, alerting on raw Count > 0 pages forever after a single historical event. Alert on OneMinuteRate > 0 or delta(Count) > 0 over an interval.
If your cluster runs the default unclean.leader.election.enable=false, this metric should always be zero. You will see OfflinePartitionsCount rise instead when no eligible leader exists. A nonzero value here means the setting was true at the time of election.
flowchart TD
A[Leader lost] --> B{ISR member available?}
B -->|Yes| C[Clean leader election]
B -->|No| D{unclean.leader.election.enable?}
D -->|False| E[Partition goes offline]
D -->|True| F[Unclean leader elected]
F --> G[Acknowledged data truncated]Common causes
| Cause | What it looks like | First thing to check |
|---|---|---|
unclean.leader.election.enable=true | UncleanLeaderElectionsPerSec fires when a leader is lost and no ISR member is available; OfflinePartitionsCount stays flat for those partitions | Broker and topic configuration for unclean.leader.election.enable |
| Multiple simultaneous broker failures | Several brokers are down or unreachable; many partitions lose all ISR members at once | Broker process liveness and network connectivity |
| ISR shrink cascade into single replica | IsrShrinksPerSec was sustained before the failure, leaving one sync replica; when that replica fails, no clean leader remains | Historical UnderReplicatedPartitions and IsrShrinksPerSec on the affected brokers |
Quick checks
Run JMX commands against the node that is the active controller.
# Verify there is exactly one active controller
echo "get -b kafka.controller:type=KafkaController,name=ActiveControllerCount Value" | java -jar jmxterm.jar -l localhost:9999
# Confirm unclean election rate
echo "get -b kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec OneMinuteRate" | java -jar jmxterm.jar -l localhost:9999
# Confirm cumulative count increment (compare to previous sample)
echo "get -b kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec Count" | java -jar jmxterm.jar -l localhost:9999
# Check the inverse signal: partitions with no leader
echo "get -b kafka.controller:type=KafkaController,name=OfflinePartitionsCount Value" | java -jar jmxterm.jar -l localhost:9999
# List unavailable partitions manually
kafka-topics.sh --bootstrap-server localhost:9092 --describe --unavailable-partitions
# List under-replicated partitions to see replication health
kafka-topics.sh --bootstrap-server localhost:9092 --describe --under-replicated-partitions
How to diagnose it
Validate the signal. Ensure you are looking at
OneMinuteRate > 0or a fresh increment inCount, not the raw cumulative value. Confirm the alert timing against recent broker restarts or network events.Correlate with total election volume. Check
kafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMs. Unclean elections are a subset of total elections. A spike here that aligns with the unclean metric confirms the controller was actively reassigning leadership.Identify the failure mode. If
unclean.leader.election.enableis enabled, that is the immediate enabler. If it is disabled at steady state but the metric fired, review configuration history for a temporary change during the incident. If the metric fired without the config being enabled, suspect multiple correlated broker failures and escalate.Find the affected scope. Cross-reference broker liveness,
UnderReplicatedPartitions, andIsrShrinksPerSecaround the incident time. If one broker was already sick and shrank ISRs before dying, that is the root cause. If multiple brokers failed together, look for a shared failure domain such as a rack or network segment.Assess data loss. For each affected partition, lost data equals the difference between the previous leader’s log end offset and the new unclean leader’s log end offset at election time. There is no automatic rollback. If the previous leader returns with a higher offset, it truncates its own log to match the new leader, cementing the loss.
Metrics and signals to monitor
| Signal | Why it matters | Warning sign |
|---|---|---|
UncleanLeaderElectionsPerSec (OneMinuteRate / delta(Count)) | Binary durability breach indicator | Any value above zero |
OfflinePartitionsCount | Inverse of unclean elections; partitions that stayed offline | Sustained > 0 when unclean is disabled |
UnderReplicatedPartitions | Durability window is open; fewer replicas than replication factor | Nonzero and growing, especially outside maintenance |
IsrShrinksPerSec | Velocity of ISR decay that precedes total unavailability | Sustained > 0 without matching expands |
ActiveControllerCount | Only the controller can elect leaders; loss blocks metadata recovery | Cluster-wide sum != 1 |
LeaderElectionRateAndTimeMs | Total election rate; unclean elections are included here | Spike correlated with unclean metric |
Fixes
Do not restart brokers or the controller reactively. Restarts generate controller events and can trigger additional leadership changes.
Disable unclean leader election
If unclean.leader.election.enable is true at the broker or topic level, disable it immediately through your configuration management pipeline. Revert any temporary enablement. Tradeoff: partitions that lose their entire ISR will become unavailable rather than sacrifice data. That is the correct posture for any cluster where durability matters. Do not re-enable it to “fix” offline partitions.
Recover from broker failures
If the trigger was multiple broker failures, restore the failed nodes before they are permanently replaced. Once a broker with more complete data returns after an unclean election, it sees the new leader’s lower log end offset and truncates its own log to match, permanently losing the divergent records. There is no built-in reconciliation that restores the lost suffix.
If a failed broker’s storage is intact but it has not yet rejoined, bringing it back online will not repair already-truncated partitions, but it may restore ISRs for other partitions and prevent further unclean elections.
Stop the cascade
If an ISR shrink storm preceded the unclean election, address the sick broker that started the cascade. In many cases, the safest immediate action is to leave the cluster as-is, fix the configuration, and plan a slow recovery. Removing an additional broker via controlled shutdown can help only if it will not further shrink ISRs for partitions still at risk.
Preserve evidence
Capture the current value of UncleanLeaderElectionsPerSec Count, LeaderElectionRateAndTimeMs percentiles, and OfflinePartitionsCount before logs rotate. You need this timeline to communicate scope and to verify the configuration change stopped new unclean elections.
Prevention
- Keep the default.
unclean.leader.election.enable=falsehas been the default since Kafka 0.11.0.0. Audit broker and topic configurations periodically to ensure it has not been overridden. - Fix ISR shrink storms early. Sustained
IsrShrinksPerSecand growingUnderReplicatedPartitionsare the runway to either unclean elections or offline partitions. Treat them as durability incidents before the last replica fails. See Kafka ISR shrinking: IsrShrinksPerSec, flapping, and the cascade to offline. - Use
min.insync.replicasandacks=all. Withmin.insync.replicas=2andreplication.factor=3, producers fail fast when durability cannot be met rather than writing to a single broker that may later be lost. See Kafka min.insync.replicas and acks: configuring durability you actually have. - Design for failure domains. Rack-aware replication and a replication factor of at least three ensure that a single rack or broker failure does not collapse the ISR to zero.
- Alert correctly. Alert on
OneMinuteRate > 0ordelta(Count) > 0forUncleanLeaderElectionsPerSec. Never alert on the raw cumulativeCount.
How Netdata helps
- Netdata collects the controller MBean and charts both
OneMinuteRateandCount, making deltas obvious across any interval. OfflinePartitionsCounton the same timeline shows whether the cluster chose durability or availability.- Overlay
IsrShrinksPerSecandUnderReplicatedPartitionsto see the replication collapse that enabled the unclean election. - Alert on
OneMinuteRatebecoming nonzero to page on the first unclean election. - Correlate the unclean election timestamp with per-broker disk I/O, network saturation, and JVM GC pauses to identify whether the root cause was broker failure, network partition, or resource exhaustion.
Related guides
- How Kafka actually works in production: a mental model for operators: /guides/kafka/how-kafka-works-in-production/
- Kafka ISR shrinking: IsrShrinksPerSec, flapping, and the cascade to offline: /guides/kafka/kafka-isr-shrink-storm/
- Kafka LEADER_NOT_AVAILABLE: causes during elections, restarts, and topic creation: /guides/kafka/kafka-leader-not-available/
- Kafka min.insync.replicas and acks: configuring durability you actually have: /guides/kafka/kafka-min-insync-replicas-misconfigured/
- Kafka monitoring checklist: the signals every production cluster needs: /guides/kafka/kafka-monitoring-checklist/
- Kafka monitoring maturity model: from survival to expert: /guides/kafka/kafka-monitoring-maturity-model/
- Kafka NotEnoughReplicasException: acks=all writes rejected below min.insync.replicas: /guides/kafka/kafka-not-enough-replicas-exception/
- Kafka OfflinePartitionsCount > 0: partitions with no leader and how to recover: /guides/kafka/kafka-offline-partitions-count/
- Kafka replica MaxLag growing: slow followers and replica fetcher health: /guides/kafka/kafka-replica-fetcher-max-lag/
- Kafka UnderMinIsrPartitionCount: confirming the write path is blocked: /guides/kafka/kafka-under-min-isr-partition-count/
- Kafka UnderReplicatedPartitions > 0: the most important metric and how to clear it: /guides/kafka/kafka-under-replicated-partitions/







