II · 08 · Metrics, Signals & Observability
Source: Apache Kafka 4.4.0-SNAPSHOT (git 04bfe7d, 2026-06-15), KRaft mode. Operational guidance grounded in source code and cited benchmarks.
A Kafka broker emits hundreds of metrics; an on-call engineer needs about a dozen. This chapter separates the signal from the noise. It traces the two metric pipelines in source, the legacy Yammer/KafkaMetricsGroup path that surfaces over JMX, and the client-side Kafka Metrics now pushable to the broker via KIP-714, then enumerates the golden signals in priority order, each cited to the exact line that emits it, each annotated with what it means and when to page. The single most useful diagnostic, the per-API request-latency breakdown, gets its own dissection because the decomposition (RequestQueueTimeMs + LocalTimeMs + RemoteTimeMs + ...) localizes almost every broker-side latency incident to one subsystem. Closes with a thresholds table you can paste into an alerting rule and a 12-panel dashboard blueprint.
The two metric pipelines
Kafka has carried two metrics libraries since its early history, and a production observability stack must scrape both. The distinction matters because they have different namespaces, different export mechanisms, and different aggregation semantics.
Server-side: Yammer (Dropwizard) Metrics via JMX. Every broker/controller-internal metric, bytes, request times, ISR counts, controller state, is registered through KafkaMetricsGroup, a thin wrapper over the Yammer MetricsRegistry. The wrapper constructs the JMX MetricName by concatenating a package (the JMX domain), a type, and a name: server-common/src/main/java/org/apache/kafka/server/metrics/KafkaMetricsGroup.java:59-67 builds the string group:type=typeName,name=metricName and appends tag key/value pairs. So BytesInPerSec registered by BrokerTopicMetrics surfaces as the JMX MBean kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec. These are Gauges (instantaneous values), Meters (1/5/15-minute exponentially-weighted rates plus a count), Histograms, and Timers (a histogram of durations plus a rate). You expose them with a JMX exporter (the Prometheus JMX agent is the de-facto standard) running in-process as a Java agent.
Client-side: the org.apache.kafka.common.metrics.Metrics library. Producers, consumers, Streams, Connect, and the broker's own network layer use a second, newer metrics system (Sensors + Measurables). These live under JMX domain kafka.consumer, kafka.producer, etc. Consumer lag (records-lag-max) is here, not in the Yammer tree. Historically these were only visible on the client JVM; KIP-714 changes that (below).
KafkaMetricsGroup → Yammer MetricsRegistry (Gauge / Meter / Histogram / Timer)domain:type=…,name=…,<tags>, scraped by Prometheus JMX agent / Datadog / Jolokiaorg.apache.kafka.common.metrics.Metrics (Sensor / Measurable) → kafka.consumer.* JMX or pushed to broker via KIP-714BrokerTopicMetrics was physically moved to the org.apache.kafka.storage.log.metrics package (KAFKA-16972), but its constructor still passes new KafkaMetricsGroup("kafka.server", "BrokerTopicMetrics") (storage/.../metrics/BrokerTopicMetrics.java:59) so the JMX name stays kafka.server:type=BrokerTopicMetrics for backward compatibility. Likewise RequestMetrics reports under kafka.network (RequestMetrics.java:82) and LogFlushStats under kafka.log (LogSegment.java:76). Scrape by JMX name, not by guessing from the Java package.
The golden signals, in priority order
Map Kafka onto the four Google-SRE golden signals (latency, traffic, errors, saturation), but recognize that the most valuable broker metrics are a handful of near-binary health gauges plus one latency decomposition. The table below is the triage order, top rows page, lower rows inform. Every metric is cited to the source that registers it.
| # | Signal | JMX name | Source (path:line) | Alert meaning |
|---|---|---|---|---|
| 1 | Active controller | kafka.controller:type=KafkaController,name=ActiveControllerCount | QuorumControllerMetrics.java:46,108-113 | SUM across cluster must be exactly 1 |
| 2 | Offline partitions | kafka.controller:type=KafkaController,name=OfflinePartitionsCount | ControllerMetadataMetrics.java:61,125 | >0 = outage (no leader) |
| 3 | Under-replicated | kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions | ReplicaManager.scala:94,241 | >0 = degraded redundancy |
| 4 | Under-min-ISR | kafka.server:type=ReplicaManager,name=UnderMinIsrPartitionCount | ReplicaManager.scala:95,242 | >0 = acks=all writes failing |
| 5 | Unclean elections | kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec | ControllerMetadataMetrics.java:67,143-144 | any >0 = data loss |
| 6 | Request latency breakdown | kafka.network:type=RequestMetrics,name=TotalTimeMs,request=… | RequestMetrics.java:105; Request.java:328-369 | p99 rising → localize via components |
| 7 | I/O thread saturation | kafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercent | KafkaRequestHandler.scala:210,253 | →0 = request handlers saturated |
| 8 | Network thread saturation | kafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercent | SocketServer.scala:120-132 | <0.3 = network processors saturated |
| 9 | ISR churn | …ReplicaManager,name=IsrShrinksPerSec / IsrExpandsPerSec | ReplicaManager.scala:100-101,257-258 | shrink without matching expand → followers falling behind |
| 10 | Consumer lag | kafka.consumer:type=consumer-fetch-manager-metrics,name=records-lag-max | FetchMetricsRegistry.java:102 | growing trend → consumers behind |
| 11 | Traffic | …BrokerTopicMetrics,name=BytesInPerSec / BytesOutPerSec / MessagesInPerSec | BrokerTopicMetrics.java:33-35 | capacity-planning baseline; sudden drop = upstream stall |
| 12 | Log flush latency | kafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs | LogSegment.java:76-77 | rising flush time → disk pressure, correlates with LocalTimeMs |
1 · ActiveControllerCount, the cluster has exactly one brain
Each controller in the KRaft quorum registers a gauge that returns active ? 1 : 0, QuorumControllerMetrics.java:108-113. The active controller (the quorum leader) reports 1; standby controllers report 0. The invariant is therefore a cluster-wide sum:
Aggregate this metric across all controller nodes and alert on any value other than 1 that persists more than a second. Sum = 0 means no node holds leadership, metadata writes (topic create, partition reassignment, ISR updates, broker registration) stall until a new leader is elected. Sum > 1 means split-brain, but note that in KRaft this is structurally far harder than the ZooKeeper era: leadership is the Raft leadership of the __cluster_metadata log, protected by the leader-epoch/quorum machinery in Part I · 10. A transient 0 during a controller failover is normal; KRaft failover is sub-second because standbys already hold the metadata log in memory (see Part I · 11), versus the O(partitions) reload that bounded ZooKeeper-era clusters to ~200K partitions (Confluent "200K Partitions" post, empirical, Kafka 1.1.0).
Companion controller metrics that diagnose why a controller is struggling, all from QuorumControllerMetrics.java: EventQueueTimeMs (line 48), how long events wait before the single controller thread picks them up; rising values mean the controller is the bottleneck. EventQueueProcessingTimeMs (line 50), how long each event takes to apply. LastAppliedRecordLagMs (line 60), for a standby, how far its applied state trails wall-clock; a large lag means a slow standby that would be slow to take over. TimedOutBrokerHeartbeatCount (line 62) and per-broker TimeSinceLastHeartbeatReceivedMs (line 71), brokers the controller is about to fence.
2 · OfflinePartitionsCount, the outage gauge
Emitted only by the active controller (ControllerMetadataMetrics.java:125, managed by ControllerMetadataMetricsPublisher). A partition is offline when it has no leader, so producers and consumers for it get LEADER_NOT_AVAILABLE and the partition is simply down. This is the clearest "customer is affected" signal in the whole system.
Because the gauge is controller-scoped, aggregate it with SUM/MAX across controllers and trust only the active one's value (the others may report stale or zero). Causes: the last in-sync replica died, a network partition isolated all replicas of a partition, or a leader's disk failed while unclean.leader.election.enable=false (the default, see II · 06) refused to promote a lagging replica. Recovery paths in II · 07.
3 · UnderReplicatedPartitions, degraded but serving
A leader-side gauge: ReplicaManager.scala:241 sums partitions where isUnderReplicated, and the predicate is exactly isLeader && (replicationFactor − isr.size) > 0 (Partition.scala:232). So URP counts partitions this broker leads whose ISR has fewer members than the assignment. The data is still readable and (unless also under-min-ISR) writable; you have simply lost redundancy headroom.
Constant non-zero, one broker's worth almost always means a broker is down, its replicas dropped out of every ISR. Fluctuating with all brokers up means a performance problem: a follower can't keep up (slow disk, network, or GC pause on the follower), so it repeatedly crosses the replica.lag.time.max.ms threshold (default 30000 ms, ReplicationConfigs.java:55) and gets ejected, then catches up and rejoins. URP is a symptom, not a root cause (oneuptime / drdroid, empirical). Note URP excludes replicas being added by an in-flight reassignment, those show under ReassigningPartitions (ReplicaManager.scala:244) instead.
4 · UnderMinIsrPartitionCount, acks=all at risk
The most important durability gauge. ReplicaManager.scala:242 counts leader partitions where isUnderMinIsr, defined as isr.size < effectiveMinIsr (Partition.scala:237-239). When this is >0, any producer using acks=all on those partitions gets NOT_ENOUGH_REPLICAS and the write is rejected, the broker refuses to acknowledge a write it cannot make durable to min.insync.replicas copies. This is the mechanism that turns a silent redundancy loss into a loud produce failure, and it is exactly why you set min.insync.replicas=2 with RF=3: see Part I · 08 and II · 06.
effectiveMinIsr is min(config.minInSyncReplicas, remoteReplicas + 1), Partition.scala:246-248. It caps your configured min.insync.replicas to the partition's actual replica count. So a topic created with RF=1 but min.insync.replicas=2 has an effective min-ISR of 1 and happily accepts acks=all writes with zero redundancy, the metric will read 0 and you get no warning. Always verify min.insync.replicas ≤ replication.factor per topic (Conduktor, empirical). The related AtMinIsrPartitionCount (ReplicaManager.scala:243) counts partitions sitting exactly at the threshold, one more failure away from rejecting writes; it is your early-warning gauge.
5 · UncleanLeaderElectionsPerSec, the data-loss counter
A Meter on the active controller (ControllerMetadataMetrics.java:143-144, type ControllerStats). It ticks whenever a partition was forced to elect an out-of-sync replica as leader because no in-sync replica was available. The new leader's log is shorter than the old committed high-watermark, so committed records past its log end are lost forever, and the partition is set to RECOVERING with ISR reset to that singleton.
With the modern default unclean.leader.election.enable=false this meter stays at 0 unless an operator deliberately enables unclean election to restore availability at the cost of durability. The default was TRUE before 0.11.0, a version-dependent footgun that caused real, recovered-only-by-luck data loss (Datadog post-mortem, empirical). Alert on rate > 0 and treat each tick as an incident.
Signal 6 · The request-latency breakdown, your single best diagnostic
If you instrument one thing well, make it this. For every API key, RequestMetrics (RequestMetrics.java:89-119) registers a family of biased histograms under kafka.network:type=RequestMetrics,name=<component>,request=<api>. The components are not independent measurements, they are consecutive segments of one request's lifetime, computed in Request.updateRequestMetrics (server/.../network/Request.java:325-337) as differences between timestamps captured as the request flows through the broker. TotalTimeMs is the sum of the segments:
TotalTimeMs ≈ RequestQueueTimeMs + LocalTimeMs + RemoteTimeMs + ThrottleTimeMs + ResponseQueueTimeMs + ResponseSendTimeMs. Because each is a separate percentile distribution, a spike in TotalTimeMs p99 can be attributed to exactly one stage, that attribution is what makes this the workhorse metric.
The request= tag is the API name (Produce, Fetch, ApiVersions, …) with two important overrides built in Request.java:339-356: a Fetch is split into FetchConsumer (client reads) and FetchFollower (replication reads) based on whether the request came from a follower, so you can separate consumer-facing latency from replication latency on the same broker. Always watch Produce, FetchConsumer, and FetchFollower separately. The diagnostic map:
| Dominant component | What it measures (Request.java) | Likely root cause | First action |
|---|---|---|---|
RequestQueueTimeMs | time waiting in the request queue before a handler picks it up (requestDequeue − start, line 328) | too few I/O threads / handlers saturated | check RequestHandlerAvgIdlePercent; raise num.io.threads |
LocalTimeMs | handler processing on this broker (apiLocalComplete − requestDequeue, line 332) | disk/page-cache pressure, GC pause, lock contention, message conversion | check LogFlushRateAndTimeMs, GC logs, MessageConversionsTimeMs |
RemoteTimeMs | time blocked in purgatory waiting on other brokers (responseComplete − apiLocalComplete, line 333), Produce: ISR acks; Fetch: data to accumulate | slow followers (Produce acks=all); or just fetch.max.wait.ms for an idle consumer | correlate with URP / IsrShrinks; for Produce, a follower is slow |
ThrottleTimeMs | quota throttling delay (line 366), not part of processing | client exceeded a produce/fetch/request quota | see Part I · 19 / II · 05; raise quota or fix client |
ResponseQueueTimeMs | response waiting for a network thread (responseDequeue − responseComplete, line 334) | too few network threads | check NetworkProcessorAvgIdlePercent; raise num.network.threads |
ResponseSendTimeMs | time to write the response onto the socket (line 335) | slow/distant client, TCP backpressure, huge response | client-side or network; check response size |
For consumer fetches, RemoteTimeMs includes the time the fetch sat in the fetch purgatory waiting for fetch.min.bytes to accumulate or fetch.max.wait.ms (default 500 ms) to elapse, see the fetch path, Part I · 09. A consumer with a low arrival rate will show large FetchConsumer RemoteTimeMs by design; it is not a problem. For Produce, by contrast, RemoteTimeMs is pure replication wait and a genuine slow-follower indicator. Two extra histograms exist only for Fetch/Produce (RequestMetrics.java:109-116): MessageConversionsTimeMs (down-conversion CPU for old clients, drive it to zero by upgrading clients) and TemporaryMemoryBytes (compression/conversion scratch memory).
Signals 7–9 · Saturation and ISR churn
RequestHandlerAvgIdlePercent, are the I/O threads keeping up?
The request-handler (I/O) thread pool services every request after the network thread hands it off. Each handler thread, in its run loop, measures the time it spends blocked waiting for the next request and marks it into a shared Meter (KafkaRequestHandler.scala:115-123). The metric is a fraction in [0, 1]: 1.0 = threads are entirely idle (healthy), 0.0 = threads never wait because work is always queued (saturated). Critically, the recorded idle time is divided by the thread count (idleTime / aggregateThreads.get, line 123) so the meter reads as a per-thread average fraction regardless of pool size, the comment at lines 111-114 explains why.
Healthy is roughly 0.6–0.8. <0.2 → the pool is a bottleneck; raise num.io.threads or add brokers. <0.1 → active saturation, requests are queuing (you'll see it in RequestQueueTimeMs). The 0.2 figure is Instaclustr/community guidance, not a code constant. Caveat: this metric has a history of being mis-reported (KAFKA-7295 calculation bug; some integrations export it as a rate; KIP-1207 fixes an anomaly in KRaft combined mode where broker and controller share threads). Confirm it reads as a 0–1 fraction in your stack before trusting thresholds.
NetworkProcessorAvgIdlePercent, are the network threads keeping up?
The SocketServer gauge (SocketServer.scala:120-132) averages the io-wait-ratio of every data-plane network processor, each processor's fraction of time blocked in select() waiting for socket I/O, clamped to ≤1.0 and averaged across processors (with an empty-pool fallback of 1.0). Same semantics as the handler idle ratio: closer to 1 is better. Confluent's guidance is to keep it above ~0.4; below ~0.3 raise num.network.threads (empirical). Companion gauges from the same registration block: MemoryPoolAvailable / MemoryPoolUsed (line 134-135) for the request-buffer memory pool, and ExpiredConnectionsKilledCount (line 136).
IsrShrinks / IsrExpands, replication health, second derivative of URP
Two Meters on each broker (ReplicaManager.scala:257-258). In steady state both should be ~0. A shrink fires when a follower crosses replica.lag.time.max.ms and is removed from an ISR this broker leads; an expand fires when it catches up and rejoins. The useful pattern is the imbalance: shrinks without matching expands mean replicas are leaving and not coming back (a broker is genuinely down or a follower is persistently slow). A steady drumbeat of matched shrink/expand pairs means a follower is oscillating around the lag threshold, a slow-host or GC-pause signal. FailedIsrUpdatesPerSec (line 259) ticking means the leader's attempts to commit an ISR change to the controller are failing, investigate controller health.
replica.lag.time.max.ms default (ReplicationConfigs.java:55) warn = lagging err = redundancy/data lostPurgatory, the latency-vs-load lens, not an alert
Delayed operations (a Produce waiting for acks=all, a Fetch waiting for fetch.min.bytes) park in a DelayedOperationPurgatory, which registers two gauges tagged by operation type (server-common/.../purgatory/DelayedOperationPurgatory.java:97-99): kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation={Produce|Fetch|...} and NumDelayedOperations. PurgatorySize counts watched operations (including some already completed but not yet purged); NumDelayedOperations is the count still pending.
With acks=all, every produce that is waiting for follower acknowledgement sits in the produce purgatory by design, so the size is expected to be non-zero (Confluent/Datadog, empirical). Use purgatory size as a diagnostic lens alongside RemoteTimeMs: a Produce purgatory growing together with Produce RemoteTimeMs confirms replication is the latency source. Alert on the symptom (latency, URP), not on raw purgatory size.
Traffic, flush, and election rate
BrokerTopicMetrics (storage/.../metrics/BrokerTopicMetrics.java:33-55) is the throughput family, available both cluster-wide (untagged) and per-topic (tagged topic=…, lazily created on first traffic, line 357-359). The capacity-planning inputs (II · 04): BytesInPerSec, BytesOutPerSec, MessagesInPerSec. Replication-specific copies exist only at the broker level: ReplicationBytesInPerSec / ReplicationBytesOutPerSec (lines 95-96, registered only when the tag set is empty), plus ReassignmentBytesInPerSec/OutPerSec for live reassignment traffic. Error meters worth watching: FailedProduceRequestsPerSec and FailedFetchRequestsPerSec (lines 39-40), and the data-quality meters InvalidMessageCrcRecordsPerSec / NoKeyCompactedTopicRecordsPerSec (lines 52-54) that flag corruption or misconfigured compacted topics. If tiered storage is on, this class also exposes the remote-copy/fetch lag and rate gauges (lines 101-119), see Part I · 05.
LogFlushRateAndTimeMs (a Timer, LogSegment.java:76-77, JMX kafka.log:type=LogFlushStats) records how long fsync takes. Rising flush time is an early disk-pressure indicator that propagates into Produce LocalTimeMs and depresses RequestHandlerAvgIdlePercent, a classic correlated trio. Note: with default settings Kafka relies on the OS page cache and does not fsync on every write (the acks=all durability boundary is "in every replica's page cache," not "on disk", a critical nuance covered in II · 06).
LeaderElectionRateAndTimeMs is gone in KRaft, don't build a panel on itThe ZooKeeper-era timer kafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMs is not emitted in 4.4. A source-wide grep finds it only inside the migration/monitoring docs (docs/getting-started/zk2kraft.md, docs/operations/monitoring.md), not in any live registration, the KRaft controller code in QuorumControllerMetrics/ControllerMetadataMetrics never declares it. In KRaft, leader election is just a metadata operation applied by the controller, so the equivalent signals are the controller event-queue timers (EventQueueTimeMs/EventQueueProcessingTimeMs, QuorumControllerMetrics.java:48,50), the unclean-election meter (signal 5), and the new ElectionFromEligibleLeaderReplicasPerSec meter for ELR promotions (KIP-966, ControllerMetadataMetrics.java:69-70,145-146). If you migrated a dashboard from a ZooKeeper cluster, replace that panel with controller-queue latency, leave it in and it is a permanently-flat dead tile.
Consumer lag, alert on the trend, from two vantage points
Lag is the gap between the log end offset and a consumer group's committed offset. There are two ways to measure it, and a robust setup uses both.
Client-side, live: the consumer's own org.apache.kafka.common.metrics tree exposes records-lag-max (FetchMetricsRegistry.java:102, JMX group consumer-fetch-manager-metrics), the maximum lag across the partitions this consumer owns, plus per-partition records-lag / records-lag-avg (lines 135-139) and the symmetric records-lead-min (line 104) telling you how close you are to the log start (and thus to losing un-consumed data to retention). The catch: a dead consumer reports nothing, so client-side lag alone cannot detect a stalled group.
External, vantage-independent: query committed offsets via the Admin API / kafka-consumer-groups.sh --describe, or run a dedicated monitor (LinkedIn's Burrow classifies each group OK/WARN/ERR by evaluating the lag trend over a sliding window, no per-topic threshold tuning; kafka_exporter is the common Prometheus alternative). This catches the stalled-consumer case the client metric misses. Group-coordination internals in Part I · 13.
A static threshold ("lag > 1M") fires on every legitimate backfill and stays quiet during a slow bleed. Alert when lag is growing over a sustained window (commonly 5+ minutes / 3 datapoints). Recovery-time mental model: time_to_drain ≈ current_lag / (consume_rate − produce_rate), if the denominator is ≤0 the group will never catch up and you must add consumers (bounded by partition count, see II · 03) or speed up processing.
Client telemetry push, KIP-714
KIP-714 closes a long-standing observability gap: client-side metrics (producer batch sizes, consumer fetch latency, lag) historically lived only on the client JVM, invisible to the cluster operator. The broker now runs a ClientMetricsManager (server/.../ClientMetricsManager.java) that lets clients push their metrics to the broker, which forwards them to a pluggable exporter. The flow: a client sends GetTelemetrySubscriptions to learn which metrics the operator wants (configured via dynamic client.metrics resources, a subscription names metric prefixes, a push interval, and a match filter), then periodically sends compressed PushTelemetry requests; the broker decodes them (supported codecs: zstd, lz4, gzip, snappy, ClientMetricsManager.java:87-88) and hands them to a ClientTelemetryExporterPlugin.
The manager keeps a per-broker LRU cache of up to 16,384 client instances (CACHE_MAX_SIZE, line 90) with a 60-second default expiry (line 91), and instruments itself via the Kafka Metrics group client-metrics (ClientMetricsStats, lines 569-625): instance-count (active client instances), unknown-subscription-request, throttle, plugin-export + plugin-export-time-avg/max (exporter latency), and plugin-error. Watch plugin-error and plugin-export-time-max to confirm telemetry is actually flowing. Architecture context in Part I · 19.
You no longer need every application team to wire up their own JMX scraping to give you consumer lag or producer retry rates, a correctly subscribed cluster collects them centrally. It does not replace broker JMX; the golden signals above are still server-side Yammer metrics. Treat KIP-714 as the channel for fleet-wide client visibility, especially for clients you don't operate.
Thresholds reference
Paste-ready alert rules. "Page" = wake someone; "Ticket" = investigate next business hour; "Watch" = dashboard only. Idle-ratio and lag thresholds are community/empirical (no compiled constant); health-gauge thresholds are structural.
| Metric | Condition | Severity | Why (mechanism) |
|---|---|---|---|
SUM(ActiveControllerCount) | ≠ 1 for > 1 s | Page | 0 = no metadata writes; >1 = split-brain (structural invariant) |
OfflinePartitionsCount | > 0 | Page | partitions have no leader = unavailable |
UnderMinIsrPartitionCount | > 0 | Page | acks=all writes rejected with NOT_ENOUGH_REPLICAS |
UncleanLeaderElectionsPerSec | > 0 | Page | committed data lost (default off; on = deliberate) |
UnderReplicatedPartitions | > 0 sustained (> 5 min) | Page / Ticket | redundancy lost; constant = broker down, fluctuating = slow follower |
RequestHandlerAvgIdlePercent | < 0.2 / < 0.1 | Ticket / Page | I/O threads saturated → RequestQueueTimeMs climbs |
NetworkProcessorAvgIdlePercent | < 0.3 | Ticket | network threads saturated → ResponseQueueTimeMs climbs |
TotalTimeMs p99 (Produce / FetchConsumer) | > SLO (workload-specific) | Ticket | latency regression; decompose via components |
IsrShrinksPerSec | > 0 without matching expand | Ticket | follower falling behind / broker down |
| Consumer lag (records-lag-max or external) | growing trend over 5+ min | Ticket / Page | consumers cannot keep up; trend > static threshold |
LogFlushRateAndTimeMs p99 | rising vs baseline | Watch | disk pressure; precedes LocalTimeMs regression |
FailedProduceRequestsPerSec / FailedFetchRequestsPerSec | > 0 sustained | Ticket | client errors or broker rejection |
EventQueueProcessingTimeMs p99 (controller) | rising vs baseline | Watch | controller is the bottleneck for metadata ops |
| Disk used (host metric) | ≥ 85% | Page | full disk → broker halts ungracefully (II · 07) |
| Heap after GC / GC pause p99 (JVM) | > 60% / > 200 ms | Ticket / Page | GC pauses cause ISR drops and timeouts (II · 05) |
ActiveControllerCount and OfflinePartitionsCount are meaningful only on the active controller; the standbys report 0 or stale values. If you alert on a per-node value you will both miss real events and fire false ones. Always aggregate across controller nodes before evaluating the threshold.
The 12-panel dashboard blueprint
One screen, top-left = most urgent, reading order = triage order. Every panel maps to a metric established above so the on-call path is "panel lights up → known runbook."
| # | Panel | Metric(s) | Viz | Reading |
|---|---|---|---|---|
| 1 | Cluster health gauges | ActiveControllerCount (sum), OfflinePartitionsCount, UnderReplicatedPartitions, UnderMinIsrPartitionCount | 4 single-stat tiles, red on non-target | all green = cluster nominal |
| 2 | Data-loss watch | UncleanLeaderElectionsPerSec, FailedIsrUpdatesPerSec | single-stat, red on >0 | any red = incident |
| 3 | Throughput | BytesInPerSec, BytesOutPerSec, MessagesInPerSec | time series | baseline; sudden drop = upstream stall |
| 4 | Produce latency breakdown | Produce {RequestQueue, Local, Remote, ResponseQueue, ResponseSend} p99 | stacked area | which segment dominates → root cause |
| 5 | Consumer-fetch latency breakdown | FetchConsumer components p99 | stacked area | high Remote often normal (fetch.max.wait) |
| 6 | Replication-fetch latency | FetchFollower components p99 + ReplicationBytesInPerSec | line + area | replication health, separate from consumers |
| 7 | Thread saturation | RequestHandlerAvgIdlePercent, NetworkProcessorAvgIdlePercent | line, 0–1, threshold bands at 0.3/0.2 | dipping toward 0 = add capacity |
| 8 | ISR churn | IsrShrinksPerSec, IsrExpandsPerSec | line | shrink ≫ expand = followers behind |
| 9 | Purgatory & queues | PurgatorySize{Produce,Fetch}, RequestQueueSize, ResponseQueueSize | line | queue backlog confirms saturation |
| 10 | Disk & flush | LogFlushRateAndTimeMs p99, disk used %, LogFlushRate | line + gauge | flush time up + disk >85% = act |
| 11 | Controller internals | EventQueueTimeMs, EventQueueProcessingTimeMs, LastAppliedRecordLagMs | line | controller bottleneck / slow standby |
| 12 | Consumer lag | per-group lag (external/admin) + records-lag-max | line per critical group | positive slope = falling behind |
Pitfalls and version caveats
- Scrape both metric trees. Yammer/JMX for broker internals; the Kafka Metrics tree (
kafka.consumer.*/ KIP-714 push) for client signals. Consumer lag is not in the broker JMX tree. RequestHandlerAvgIdlePercenthas a buggy lineage. KAFKA-7295 (calculation), Datadog integrations-core (rate-vs-gauge), KIP-1207 (KRaft combined-mode anomaly where one thread pool serves broker and controller roles). Verify it reads 0–1 before alerting.- Health gauges are controller-scoped.
OfflinePartitionsCountandActiveControllerCountmust be aggregated; per-node values mislead. - Non-zero Produce purgatory is normal under
acks=all. Don't alert on raw purgatory size; use it to confirm aRemoteTimeMsdiagnosis. min.insync.replicasis silently capped to the replica count byeffectiveMinIsr(Partition.scala:246-248), a topic with RF <min.insync.replicasgives no warning and reads 0 under-min-ISR. Audit per topic.- ZooKeeper-era metrics are gone. KRaft removed ZooKeeper in 4.0;
outstanding_requests,pending_syncs,KafkaZookeeperSyncConnectno longer exist. Much published monitoring guidance predates KRaft, the controller-quorum metrics (EventQueueTimeMs,LastAppliedRecordLagMs) replace them. - Client-side
records-lag-maxmisses dead consumers. Pair it with an external committed-offset monitor (Burrow /kafka_exporter) that evaluates the lag trend independent of consumer liveness. - Alert on trend for noisy signals (lag, latency) and on absolute value only for the binary health gauges. Gate pages on impact and a sustained window to avoid flapping.