Kafka KRaft metadata log lag: standby controllers and brokers falling behind
When client errors like NOT_LEADER_FOR_PARTITION or LeaderNotAvailableException hit a stable-looking cluster, check whether standby controllers and broker metadata lag are climbing. In KRaft mode, the active controller is the Raft quorum leader. It appends metadata changes to __cluster_metadata. Standby controllers and brokers apply that log asynchronously. When they fall behind, they act on stale partition leadership, outdated ISR memberships, and old broker registrations. This guide shows how to distinguish a slow follower, a sick leader, and a network partition.
What this means
In KRaft mode, Kafka replaces ZooKeeper with a Raft-based metadata quorum. The active controller is the Raft leader. It writes metadata records (topic changes, leader elections, configuration updates) to the __cluster_metadata partition. Standby controllers and brokers consume this log asynchronously and apply records to their local MetadataImage.
On the active controller, kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs reads 0 because records apply as they commit. On standby controllers and brokers, last-applied-record-lag-ms should stay near zero in steady state. A sustained non-zero value means a stale cluster view. Stale brokers return outdated metadata to producers and consumers, so clients send requests to the wrong broker. Stale standbys may fail to take over promptly if the active controller fails. The root cause is usually one of three: the Raft leader commits too slowly, the network path between leader and follower is degraded, or the lagging node cannot apply records because of local saturation or corruption.
flowchart TD
A[Standby or broker metadata lag growing] --> B{Active controller lag = 0?}
B -->|No| C[Investigate quorum leader or controller event queue]
B -->|Yes| D{Lagging nodes}
D -->|Single node| E[Node saturation disk CPU GC or metadata errors]
D -->|Multiple nodes| F[Leader bottleneck or inter-node network]
E --> G[Check broker disk await and GC activity]
G --> H[Add capacity if saturated; restart only if corrupted]
F --> I[Check commit-latency-avg and EventQueueSize]
I --> J[Reduce metadata churn or fix network path]Common causes
| Cause | What it looks like | First thing to check |
|---|---|---|
| Inter-node network degradation | Lag grows uniformly on multiple standbys or brokers; no single node stands out | commit-latency-avg under kafka.server:type=raft-metrics and cross-node latency |
| Lagging node saturated | Lag isolated to one broker or standby; local disk await or CPU is high | Disk stats and JVM GC activity on the lagging node |
| Controller event queue backup | Active controller lag is 0 but queue size is high; elections and ISR updates slow | EventQueueSize on the active controller |
| Metadata load or apply errors | Lagging node shows error counts under broker-metadata-metrics | Broker logs for metadata snapshot load failures |
| DNS or startup race (containerized) | Freshly restarted controller never catches up; current-state stuck | kafka-metadata-quorum.sh describe --status and node state |
Quick checks
Run these read-only commands before making changes.
# Check quorum status, MaxFollowerLag, and HighWatermark
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --status
# Check standby controller metadata lag via JMX
echo "get -b kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs Value" | java -jar jmxterm.jar -l localhost:9999
# Check broker metadata lag via JMX
echo "get -b kafka.server:type=broker-metadata-metrics,name=last-applied-record-lag-ms Value" | java -jar jmxterm.jar -l localhost:9999
# Check Raft commit latency on the leader
echo "get -b kafka.server:type=raft-metrics commit-latency-avg" | java -jar jmxterm.jar -l localhost:9999
# Check disk latency on the lagging node
iostat -xz 1
# Check GC activity on the lagging broker
jstat -gcutil $(pgrep -f kafka.Kafka) 1000
pgrep -f kafka.Kafka returns multiple PIDs if more than one Java process matches, which breaks jstat. Use the exact broker PID when multiple JVMs run on the host.
How to diagnose it
Confirm the active controller is healthy. Query
LastAppliedRecordLagMson every controller-eligible node. Exactly one node should report0; that is the active leader. If the active controller itself is lagging or if no node reports0, you have a quorum problem, not a follower problem. See Kafka ActiveControllerCount not equal to 1: no controller or split brain.Map the scope of the lag. Use
kafka-metadata-quorum.sh describe --statusto see which voters and observers are behind. If the lag is isolated to a single standby or broker, suspect node-level saturation or a local disk issue. If multiple nodes lag together, suspect network degradation or an overwhelmed Raft leader.Inspect the Raft leader’s commit path. On the active controller, check
commit-latency-avgandcommit-latency-maxunderkafka.server:type=raft-metrics. Elevated commit latency (sustained above tens of milliseconds) indicates the leader is struggling to replicate metadata records. Correlate withEventQueueSize. A backed-up controller event queue slows metadata changes before they reach the Raft log.Inspect the lagging node’s ability to apply records. On a lagging broker or standby, check disk I/O
awaitand JVM GC pause times. A node that cannot apply metadata because it is blocked on local disk or garbage collection will show growing lag even if the network is fine. Also check broker logs for snapshot load or apply failures.Check for client impact. If broker metadata lag is nonzero, expect
FailedProduceRequestsPerSecandFailedFetchRequestsPerSecto rise, includingNOT_LEADER_FOR_PARTITIONerrors. Cross-reference with consumer and producer logs to confirm stale metadata is causing retries.Verify the fix is taking hold. After addressing the root cause, watch
last-applied-record-lag-msreturn to baseline. Do not declare the incident resolved until lag has been near zero for several consecutive samples.
Metrics and signals to monitor
| Signal | Why it matters | Warning sign |
|---|---|---|
kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs | Measures how far behind a standby controller is on the metadata log | Non-zero and growing on any standby |
kafka.server:type=broker-metadata-metrics,name=last-applied-record-lag-ms | Measures broker metadata staleness; stale brokers mislead clients | Sustained value above baseline or nonzero during steady state |
kafka.server:type=raft-metrics commit-latency-avg / commit-latency-max | Reveals whether the Raft leader can commit metadata records efficiently | Sustained values above tens of milliseconds |
kafka.server:type=raft-metrics current-state | Shows whether a node is attached to the quorum as leader or follower | Values other than leader or follower indicate detachment |
EventQueueSize | Backed-up controller events delay metadata changes before they hit the log | Consistently above 100, or growing without bound |
metadata-load-error-count / metadata-apply-error-count | Direct evidence that a node is failing to process metadata | Any non-zero value in steady state |
Disk await (OS) | Local disk saturation prevents metadata snapshot application | await above 20 ms for SSDs or 50 ms for HDDs sustained |
| JVM GC pause time | Long pauses stall the thread applying metadata records | Full GC pauses above 200 ms or Young GC above 50 ms regularly |
Fixes
Network degradation between quorum members. If commit-latency-avg is elevated and multiple followers lag, check inter-node packet loss and latency. In cloud environments, ensure controller nodes are in the same region or availability zone with low RTT. If a network partition is confirmed, fix the routing or security group issue. Do not restart controllers during a suspected partition; this can trigger unnecessary leader elections.
Saturated lagging node. If the lag is isolated to one broker or standby, treat it as a local resource problem. Check whether disk I/O await is high, whether GC pauses are stalling the broker, or whether RequestHandlerAvgIdlePercent shows the node is overloaded. For disk saturation, move traffic away if possible or add I/O capacity. For GC issues, verify the heap is not oversized and that message format down-conversion is not materializing large on-heap buffers.
Controller event queue backup. If the active controller’s event queue is growing, the leader is overwhelmed by metadata churn. Stop any non-essential admin operations (topic creation, configuration changes, partition reassignments) until the queue drains. In large clusters, avoid mass rolling restarts that generate thousands of simultaneous leadership events. See Kafka controller event queue backing up: overwhelmed controller and stalled metadata.
Persistent metadata corruption on a follower. If a node shows metadata load or apply errors and lag never recovers, the local metadata snapshot or log may be corrupt. A controlled rolling restart of the affected node will force it to fetch a fresh snapshot from the leader. This is disruptive: the node will lose its page cache and must replay state. Only do this after ruling out network and resource issues. In containerized environments with persistent volumes, verify the volume is healthy before restart.
Quorum wedging after startup. If a freshly provisioned controller never enters the quorum and current-state is not follower or leader, investigate DNS resolution and startup ordering. Some operators have seen controllers wedge because hostnames are not resolvable at JVM start time . Recovery may require rolling the controllers one by one, leader last. If your orchestrator relies on port-only readiness probes, consider adding a metadata-state probe.
Prevention
Monitor LastAppliedRecordLagMs and last-applied-record-lag-ms as first-class signals. A metadata lag alert should fire before client errors appear. Keep controller nodes in a low-latency, highly reliable network segment. Do not run combined broker-controller mode in production; it couples the data plane and control plane recovery paths and makes attribution harder. Test controller failover and broker restart times in staging so you know how long metadata replay takes with your partition count. Patch Kafka regularly to pick up KRaft stability fixes.
How Netdata helps
- Correlates
last-applied-record-lag-mswith the same node’s diskawait, CPU, and JVM GC metrics in one view, which helps distinguish local saturation from quorum-wide issues. - Tracks
commit-latency-avgalongsideEventQueueSizeso you can see whether the bottleneck is Raft replication or controller event processing. - Alerts on metadata lag growth rates rather than static thresholds, catching drift before clients see
NOT_LEADER_FOR_PARTITION. - Surfaces broker error rates (
FailedProduceRequestsPerSec,FailedFetchRequestsPerSec) on the same dashboard as metadata lag to confirm client impact.
Related guides
- How Kafka actually works in production: a mental model for operators: /guides/kafka/how-kafka-works-in-production/
- Kafka controller event queue backing up: overwhelmed controller and stalled metadata: /guides/kafka/kafka-controller-event-queue-backup/
- Kafka ISR shrinking: IsrShrinksPerSec, flapping, and the cascade to offline: /guides/kafka/kafka-isr-shrink-storm/
- Kafka LeaderElectionRateAndTimeMs spiking: election storms and slow elections: /guides/kafka/kafka-leader-election-rate-high/
- Kafka LEADER_NOT_AVAILABLE: causes during elections, restarts, and topic creation: /guides/kafka/kafka-leader-not-available/
- Kafka leadership imbalance: LeaderCount skew and preferred replica election: /guides/kafka/kafka-leadership-imbalance/
- 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 ActiveControllerCount not equal to 1: no controller or split brain: /guides/kafka/kafka-no-active-controller/
- Kafka NotEnoughReplicasException: acks=all writes rejected below min.insync.replicas: /guides/kafka/kafka-not-enough-replicas-exception/
- Kafka NOT_LEADER_FOR_PARTITION: stale metadata, controller lag, and client retries: /guides/kafka/kafka-not-leader-for-partition/







