Version & Environment
Redpanda version: v26.1.4 (commit 076e88a8e87668341e966bdfe2755a9dcbcf4a7f)
- Kubernetes: server
v1.32.10, client v1.31.0
- Deployed via Redpanda Operator on self-hosted Kubernetes on AWS (
r5a.2xlarge EC2 instances, EBS-backed NVMe via Nitro)
- 3-broker StatefulSet, RF=3, SASL+TLS enabled
- OS: Linux
6.16.4-zabbly+ (Ubuntu 24.04, #ubuntu24.04 SMP PREEMPT_DYNAMIC Fri Aug 29 00:54:49 UTC 2025 x86_64)
- Storage: EBS gp3, XFS filesystem
- No Kafka client libraries — issue is observed via
rpk and Admin API only
What went wrong?
rpk cluster health, /v1/cluster/health_overview, the Redpanda Console UI, and Prometheus metrics (redpanda_kafka_under_replicated_replicas) all consistently report leaderless and under-replicated partitions when the cluster is actually healthy.
This has happened repeatedly (at least 3 times in 2 days) and persists for extended periods (many hours) without resolving. All three brokers are up and running throughout.
Example output from rpk cluster health while the issue is occurring:
CLUSTER HEALTH OVERVIEW
=======================
Healthy: false
Unhealthy reasons: [leaderless_partitions under_replicated_partitions]
Controller ID: 2
All nodes: [0 1 2]
Nodes down: []
Nodes in recovery mode: []
Nodes with high disk usage: []
Leaderless partitions (16): [kafka/__consumer_offsets/0 kafka/__consumer_offsets/5 kafka/__consumer_offsets/8 kafka/_schemas/0 kafka/block.io.req.v1/0 kafka/fdb.trace-logs.v1/0 kafka/fdb.trace-logs.v1/2 kafka/fdb.trace-logs.v1/4 kafka/fdb.trace-logs.v1/6 kafka/fdb.trace-logs.v1/7 kafka/fdb.trace-logs.v1/8 kafka/k8s.events/0 kafka/mercury.v1/0 kafka/mercury.v4/0 kafka/owl.enriched.full/0 kafka/pg.log_metadata.v2/0]
Under-replicated partitions (8): [kafka/__consumer_offsets/0 kafka/__consumer_offsets/10 kafka/__consumer_offsets/14 kafka/__consumer_offsets/2 kafka/__consumer_offsets/5 kafka/__consumer_offsets/7 kafka/__consumer_offsets/8 kafka/fdb.trace-logs.v1/4]
At the same time, /v1/cluster/health_overview also reports is_healthy: false with the same leaderless/under-replicated lists.
However, querying individual partitions via the Admin API shows all partitions have valid leaders:
$ kubectl exec -n savannah-system redpanda-0 -c redpanda -- \
curl -sk "https://localhost:9644/v1/partitions/kafka/owl.enriched.full/0" | python3 -m json.tool
{
"ns": "kafka",
"topic": "owl.enriched.full",
"partition_id": 0,
"status": "done",
"leader_id": 2,
"raft_group_id": 58,
"replicas": [
{"node_id": 1, "core": 0},
{"node_id": 2, "core": 0},
{"node_id": 0, "core": 0}
],
"disabled": false
}
We verified this for every partition reported as leaderless — all returned a valid leader_id and status: done.
As definitive proof, we produced directly to four of the "leaderless" topics while the health endpoints were still reporting them as leaderless. All produces succeeded immediately with no errors:
=== fdb.trace-logs.v1 ===
Produced to partition 2 at offset 0 with timestamp 1778079854916.
=== owl.enriched.full ===
Produced to partition 0 at offset 1 with timestamp 1778079857260.
=== block.io.req.v1 ===
Produced to partition 0 at offset 0 with timestamp 1778079865262.
=== pg.log_metadata.v2 ===
Produced to partition 0 at offset 0 with timestamp 1778079867639.
A genuinely leaderless partition would return LEADER_NOT_AVAILABLE immediately — these all succeeded, confirming the health reporting is stale.
The Redpanda Console UI shows the same stale data under Cluster Health Debug:
Reason: Leaderless partitions, under-replicated partitions
Unreachable brokers: 0
Leaderless partitions: 16 View topics
Under-replicated partitions: 11 View topics
This confirms that the Console is reading from the same stale health cache — not independently verifying partition state.
Impact: the Redpanda Operator sidecar container uses the health endpoint for its readiness probe. When the health cache is stale, all broker pods show 1/2 (sidecar not ready), causing unnecessary alerts and operational confusion.
The sidecar logs confirm it is reacting to the stale health state:
{"level":"info","ts":"2026-05-06T14:26:48.267Z","logger":"Prober","msg":"broker has leaderless or under-replicated partitions","leaderless":0,"under-replicated":1}
All three broker pods (redpanda-0, redpanda-1, redpanda-2) show 1/2 simultaneously:
NAME READY STATUS RESTARTS AGE
redpanda-0 1/2 Running 0 11h
redpanda-1 1/2 Running 0 11h
redpanda-2 1/2 Running 1 (11h ago) 23h
What should have happened instead?
rpk cluster health and /v1/cluster/health_overview should reflect the actual partition state in real time, consistent with what /v1/partitions/kafka/<topic>/<partition> returns. If all partitions have valid leaders and all replicas are in sync, the cluster should report Healthy: true.
At minimum, the staleness window should be short (seconds, not tens of minutes), and there should be a mechanism to force a cache refresh.
How to reproduce the issue?
We observed this consistently after periods of produce/consume activity. A possible reproduction path:
-
Deploy a 3-broker Redpanda cluster (RF=3, SASL+TLS) via the Redpanda Operator on Kubernetes
-
Create a test topic with RF=3:
kubectl exec -n savannah-system redpanda-0 -c redpanda -- \
rpk topic create test-grafana --partitions 3 --replicas 3
- Produce a burst of ~30,000 messages:
kubectl exec -n savannah-system redpanda-0 -c redpanda -- \
bash -c 'for i in $(seq 1 30000); do echo "message-$i payload-$(cat /dev/urandom | tr -dc a-z | head -c 200)"; done | rpk topic produce test-grafana --compression snappy'
- Consume all messages with a named consumer group (triggers
__consumer_offsets commit activity):
kubectl exec -n savannah-system redpanda-0 -c redpanda -- \
rpk topic consume test-grafana --group test-grafana-consumer --num 30000
- Observe
rpk cluster health — it reports leaderless and under-replicated partitions across many topics (not just test-grafana):
kubectl exec -n savannah-system redpanda-0 -c redpanda -- rpk cluster health
- At the same time, query individual partitions via the Admin API — they show valid leaders:
kubectl exec -n savannah-system redpanda-0 -c redpanda -- \
curl -sk "https://localhost:9644/v1/partitions/kafka/owl.enriched.full/0" | python3 -m json.tool
- Produce directly to one of the "leaderless" topics — it succeeds immediately with no error:
echo "health-test" | kubectl exec -i -n savannah-system redpanda-0 -c redpanda -- \
rpk topic produce owl.enriched.full
The issue has occurred 3 times in 2 days on the same cluster, always with the same pattern.
Important: the stale health report includes partitions on topics that were never written to during the reproduce steps (e.g. fdb.trace-logs.v1, owl.enriched.full, pg.log_metadata.v2, k8s.events, mercury.v1). Only test-grafana was produced to. This rules out the possibility that the produce/consume activity directly caused those partitions to become unhealthy — the health cache appears to apply a cluster-wide stale view, not just to the partitions that were recently active.
Additional information
- All three observability surfaces share the same stale cache:
rpk cluster health, /v1/cluster/health_overview, the Redpanda Console UI, and Prometheus metrics (redpanda_kafka_under_replicated_replicas). They all agree with each other but are all wrong simultaneously.
- The only authoritative source is
/v1/partitions/kafka/<topic>/<partition>, which queries the Raft controller directly.
- The issue appears to be triggered by
__consumer_offsets partition churn (burst of consumer group commits), though we cannot confirm this definitively.
- The cluster is Community Edition, no enterprise features enabled (
partition_autobalancing_mode: node_add, core_balancing_continuous: false) even we are on the enterprise trial (but we explicitly disabled Enterprise features as we will use the Community edition).
- We did not attach the debug bundle as it contains internal topology information, but can provide it privately if needed.
Thank you for your help.
Version & Environment
Redpanda version:
v26.1.4(commit076e88a8e87668341e966bdfe2755a9dcbcf4a7f)v1.32.10, clientv1.31.0r5a.2xlargeEC2 instances, EBS-backed NVMe via Nitro)6.16.4-zabbly+(Ubuntu 24.04,#ubuntu24.04 SMP PREEMPT_DYNAMIC Fri Aug 29 00:54:49 UTC 2025 x86_64)rpkand Admin API onlyWhat went wrong?
rpk cluster health,/v1/cluster/health_overview, the Redpanda Console UI, and Prometheus metrics (redpanda_kafka_under_replicated_replicas) all consistently report leaderless and under-replicated partitions when the cluster is actually healthy.This has happened repeatedly (at least 3 times in 2 days) and persists for extended periods (many hours) without resolving. All three brokers are up and running throughout.
Example output from
rpk cluster healthwhile the issue is occurring:At the same time,
/v1/cluster/health_overviewalso reportsis_healthy: falsewith the same leaderless/under-replicated lists.However, querying individual partitions via the Admin API shows all partitions have valid leaders:
We verified this for every partition reported as leaderless — all returned a valid
leader_idandstatus: done.As definitive proof, we produced directly to four of the "leaderless" topics while the health endpoints were still reporting them as leaderless. All produces succeeded immediately with no errors:
A genuinely leaderless partition would return
LEADER_NOT_AVAILABLEimmediately — these all succeeded, confirming the health reporting is stale.The Redpanda Console UI shows the same stale data under Cluster Health Debug:
This confirms that the Console is reading from the same stale health cache — not independently verifying partition state.
Impact: the Redpanda Operator sidecar container uses the health endpoint for its readiness probe. When the health cache is stale, all broker pods show
1/2(sidecar not ready), causing unnecessary alerts and operational confusion.The sidecar logs confirm it is reacting to the stale health state:
All three broker pods (
redpanda-0,redpanda-1,redpanda-2) show1/2simultaneously:What should have happened instead?
rpk cluster healthand/v1/cluster/health_overviewshould reflect the actual partition state in real time, consistent with what/v1/partitions/kafka/<topic>/<partition>returns. If all partitions have valid leaders and all replicas are in sync, the cluster should reportHealthy: true.At minimum, the staleness window should be short (seconds, not tens of minutes), and there should be a mechanism to force a cache refresh.
How to reproduce the issue?
We observed this consistently after periods of produce/consume activity. A possible reproduction path:
Deploy a 3-broker Redpanda cluster (RF=3, SASL+TLS) via the Redpanda Operator on Kubernetes
Create a test topic with RF=3:
kubectl exec -n savannah-system redpanda-0 -c redpanda -- \ rpk topic create test-grafana --partitions 3 --replicas 3__consumer_offsetscommit activity):kubectl exec -n savannah-system redpanda-0 -c redpanda -- \ rpk topic consume test-grafana --group test-grafana-consumer --num 30000rpk cluster health— it reports leaderless and under-replicated partitions across many topics (not justtest-grafana):kubectl exec -n savannah-system redpanda-0 -c redpanda -- rpk cluster healthThe issue has occurred 3 times in 2 days on the same cluster, always with the same pattern.
Important: the stale health report includes partitions on topics that were never written to during the reproduce steps (e.g.
fdb.trace-logs.v1,owl.enriched.full,pg.log_metadata.v2,k8s.events,mercury.v1). Onlytest-grafanawas produced to. This rules out the possibility that the produce/consume activity directly caused those partitions to become unhealthy — the health cache appears to apply a cluster-wide stale view, not just to the partitions that were recently active.Additional information
rpk cluster health,/v1/cluster/health_overview, the Redpanda Console UI, and Prometheus metrics (redpanda_kafka_under_replicated_replicas). They all agree with each other but are all wrong simultaneously./v1/partitions/kafka/<topic>/<partition>, which queries the Raft controller directly.__consumer_offsetspartition churn (burst of consumer group commits), though we cannot confirm this definitively.partition_autobalancing_mode: node_add,core_balancing_continuous: false) even we are on the enterprise trial (but we explicitly disabled Enterprise features as we will use the Community edition).Thank you for your help.