A note for the community
- Please vote on this issue by adding a π reaction to the original issue to help the community and maintainers prioritize this request
- If you are interested in working on this issue or have submitted a pull request, please leave a comment
Problem
After a brief Kafka broker fencing event (KRaft controller GC pause β broker session timeout exceeded), Vector's Kafka sink enters a permanent stuck state where all Kafka produce operations fail with MessageTimedOut (Local: Message timed out). The stuck state persists indefinitely β no self-recovery occurs despite the Kafka broker fully recovering within 1β6 seconds of the trigger.
When rd_kafka_topic_metadata_update() processing "leader=-1 not found in cache", it sets a topic-level state that subsequent valid metadata responses don't clear
What currently happens:
- KRaft controller GC pause causes the broker to be fenced. During the fenced window (1β6 seconds in observed incidents), the broker reports
leader=-1 for all topic partitions in metadata responses.
- Vector's librdkafka processes a metadata response with
leader=-1. It cannot find broker -1 in its node cache β logs METADATAUPDATE [thrd:main]: Partition logs.syslog[N]: new leader-1 not found in cache. All partitions are assigned to librdkafka's internal broker sentinel (a non-network in-process placeholder).
- The topic-level state is set to
NO_INFO ("metadata information unknown").
- The broker recovers within seconds. librdkafka's main thread detects the
internal broker assignment and triggers metadata re-queries: QRYLEADER [thrd:main]: Topic logs.syslog [0]: broker is internal: re-query β firing every ~30 seconds matching topic.metadata.refresh.interval.ms.
- The re-queries reach the recovered broker and receive valid
leader=0 responses. However, the topic-level NO_INFO flag is never cleared. The partition scheduler checks this flag first and refuses to route messages regardless.
- All queued messages expire after
message.timeout.ms and are dropped as MessageTimedOut. Vector logs Events dropped. No re-queue to disk buffer occurs.
- Vector continues operating normally in all other respects (
/health returns 200, sources keep accepting events, disk buffer keeps growing). The failure is completely silent.
What I expected:
After the broker recovers and metadata re-queries succeed with valid leader information:
- Vector should detect the persistent stuck state and either (a) recreate the producer or (b) exit the process to allow Kubernetes to restart it and replay the disk buffer
- librdkafka should clear the
NO_INFO state and resume producing normally β consistent with how it recovers from other transient leader unavailability errors (e.g. LEADER_NOT_AVAILABLE).
This bug is distinguishable from a plain MessageTimedOut timeout issue. The stuck state persists for the entire remaining lifetime of the Vector process (confirmed across 7+ minutes in one incident). A pod restart fully recovers it.
Configuration
sinks:
kafka_logs:
type: kafka
inputs: [enrich_syslog_labels]
bootstrap_servers: "centralbus-kafka-bootstrap.centralbuffer.svc.cluster.local:9092"
topic: "logs.syslog"
encoding:
codec: json
acknowledgements:
enabled: true
librdkafka_options:
"debug": "all"
"log_level": "7"
"topic.metadata.refresh.interval.ms": "30000"
"topic.metadata.refresh.fast.interval.ms": "250"
"socket.keepalive.enable": "true"
"message.timeout.ms": "30000"
buffer:
type: disk
max_size: 10737418240
when_full: block
Kafka: Strimzi-managed KRaft cluster, single broker (broker.session.timeout.ms default 9000ms), and single controller. KRaft controller configured with 256m heap (GC pauses exceed 9s, triggering broker fencing).
Version
vector 0.55.0 (aarch64-unknown-linux-gnu cf8de83 2026-04-22 14:31:18.008404048)
Debug Output
The following log lines capture the full failure sequence. Collected with VECTOR_LOG=trace and librdkafka_options: debug: "all", log_level: "7".
The trigger event (fires once, then immediately suppressed by Vector's tracing-limit):
2026-05-06T05:47:16.229163Z WARN librdkafka: librdkafka: METADATAUPDATE [thrd:main]: Partition logs.syslog(JqRFQ33JTbeFO83vFyDZvg)[5]: new leader-1 not found in cache
2026-05-06T05:47:16.407332Z WARN log: Internal log [librdkafka: METADATAUPDATE [thrd:main]: Partition logs.syslog(JqRFQ33JTbeFO83vFyDZvg)[5]: new leader-1 not found in cache] is being suppressed to avoid flooding.
librdkafka IS re-querying (~30s interval = topic.metadata.refresh.interval.ms), but the topic stays stuck:
2026-05-06T05:48:14.463261Z DEBUG librdkafka: librdkafka: QRYLEADER [thrd:main]: Topic logs.syslog [0]: broker is internal: re-query
2026-05-06T05:48:44.594717Z DEBUG librdkafka: librdkafka: QRYLEADER [thrd:main]: Topic logs.syslog [0]: broker is internal: re-query
Topic-level NO_INFO persists indefinitely after re-queries succeed (rate-limited to 1 per 10s by Vector's tracing-limit; underlying frequency is higher):
2026-05-06T05:49:04.635255Z DEBUG librdkafka: librdkafka: NOINFO [thrd:main]: Topic logs.syslog metadata information unknown
2026-05-06T05:49:14.645009Z DEBUG librdkafka: librdkafka: NOINFO [thrd:main]: Topic logs.syslog metadata information unknown
2026-05-06T05:49:24.687674Z DEBUG librdkafka: librdkafka: NOINFO [thrd:main]: Topic logs.syslog metadata information unknown
2026-05-06T05:49:34.702318Z DEBUG librdkafka: librdkafka: NOINFO [thrd:main]: Topic logs.syslog metadata information unknown
... (continues every 10s with no recovery)
Messages being dropped continuously:
2026-05-06T05:47:46.158981Z ERROR sink{component_id=kafka_logs}: vector_common::internal_event::service: Service call failed. No retries or retries exhausted. error=Some(KafkaError (Message production error: MessageTimedOut (Local: Message timed out)))
2026-05-06T05:47:56.488686Z ERROR sink{component_id=kafka_logs}: vector_common::internal_event::service: Service call failed. No retries or retries exhausted. error=Some(KafkaError (Message production error: MessageTimedOut (Local: Message timed out)))
... (continues every ~10s indefinitely)
Individual per-partition timeouts in the internal broker thread:
2026-05-06T05:48:04.381929Z DEBUG librdkafka: librdkafka: TIMEOUT [thrd::0/internal]: :0/internal: logs.syslog [7]: timed out 0+1 message(s) (MsgId 999..999): message.timeout.ms exceeded
2026-05-06T05:48:24.476710Z DEBUG librdkafka: librdkafka: TIMEOUT [thrd::0/internal]: :0/internal: logs.syslog [2]: timed out 0+1 message(s) (MsgId 996..996): message.timeout.ms exceeded
Kafka broker side β confirmed the broker recovered before Vector's stuck state began:
2026-05-06 05:47:15 INFO [kafka-0-metadata-loader-event-handler] [Broker id=0] Transitioning 73 partition(s) to local followers.
2026-05-06 05:47:16 INFO [kafka-0-metadata-loader-event-handler] [Broker id=0] Transitioning 73 partition(s) to local leaders.
The broker was in the fenced (leader=-1) state for less than 1 second. Vector's NOINFO event landed at 05:47:16.229 β within the sub-second fencing window.
Example Data
N/A β this is a connection/metadata state failure, not a data encoding issue. The failure occurs with any payload.
Additional Context
- Vector is running in Kubernetes (single-replica Deployment,
restartPolicy: Always).
- The Kafka sink uses a 10 GiB disk buffer. The disk buffer write path remains healthy throughout β new events keep accumulating. Only the read/drain path (Kafka produce side) is stuck. The disk buffer does not re-receive dropped messages; once librdkafka drops them via
MessageTimedOut, they are gone.
- Vector's
/health endpoint returns 200 OK for the entire duration of the stuck state. The standard httpGet liveness probe does not detect this failure. A metric-based probe on vector_component_sent_events_total{component_id="kafka_logs"} would be required to auto-restart on this failure.
- The
WARN log for new leader-1 not found in cache fires only twice (first occurrence + "being suppressed") before Vector's tracing-limit layer silences it for the 10-second window. Since no further WARN-level librdkafka events occur after the window expires, the rate-limit summary (has been suppressed N times) is never printed. From the operator's perspective the logs appear to show a brief transient error that resolved β while in reality the sink is permanently broken.
- This has been observed and confirmed across 3 separate incidents from the same Kafka pod's repeated GC cycles (timestamps
02:58:18, 03:16:41, 05:47:16). In one incident the failure persisted for 7+ minutes until the pod was manually restarted.
- The bug is specific to the
leader=-1 path. Normal LEADER_NOT_AVAILABLE responses (where librdkafka finds a valid leader=N broker) appear to recover correctly.
How to reproduce the leader=-1 scenario:
The trigger is a KRaft controller JVM GC pause that exceeds broker.session.timeout.ms. We reproduce it reliably using the following setup:
Environment:
- Strimzi KRaft cluster (
kraft: {} mode), single broker + single controller pool
- Controller JVM heap intentionally constrained:
-Xms128m -Xmx128m (128m is reliable; 256m triggers it less frequently)
broker.session.timeout.ms left at the default (9000ms) β no tuning needed at this size heap
Trigger (two options):
Option A β natural (wait for GC): With a 128m controller heap and a modest produce workload running, the controller heap exhausts and triggers stop-the-world GC pauses of 10β30 seconds within minutes. No manual intervention needed β just watch the controller logs for EventPerformanceMonitor: Exceptionally slow controller event ... took NNNN ms.
Option B β forced (on-demand): Trigger a full GC on the controller JVM manually:
# Forces a full GC pause on the controller, typically 5β20s at 256m heap
kubectl exec -n <namespace> <controller-pod> -- jcmd 1 GC.run
# More aggressive β triggers GC + forces live object enumeration (longer pause)
kubectl exec -n <namespace> <controller-pod> -- jmap -histo:live 1
Run either command while Vector is actively producing to Kafka. The controller pause causes the broker heartbeat to time out β controller fences broker 0 β all partition leaders become -1 β broker immediately recovers via ELR. If Vector's 30s metadata refresh cycle lands in the brief fenced window, the bug triggers.
Confirmation the bug triggered (check Vector logs):
WARN librdkafka: METADATAUPDATE [thrd:main]: Partition <topic>[N]: new leader-1 not found in cache
After this line, all produce to that topic will silently fail until Vector is restarted.
References
A note for the community
Problem
After a brief Kafka broker fencing event (KRaft controller GC pause β broker session timeout exceeded), Vector's Kafka sink enters a permanent stuck state where all Kafka produce operations fail with
MessageTimedOut (Local: Message timed out). The stuck state persists indefinitely β no self-recovery occurs despite the Kafka broker fully recovering within 1β6 seconds of the trigger.When rd_kafka_topic_metadata_update() processing "leader=-1 not found in cache", it sets a topic-level state that subsequent valid metadata responses don't clear
What currently happens:
leader=-1for all topic partitions in metadata responses.leader=-1. It cannot find broker -1 in its node cache β logsMETADATAUPDATE [thrd:main]: Partition logs.syslog[N]: new leader-1 not found in cache. All partitions are assigned to librdkafka's internal broker sentinel (a non-network in-process placeholder).NO_INFO("metadata information unknown").internalbroker assignment and triggers metadata re-queries:QRYLEADER [thrd:main]: Topic logs.syslog [0]: broker is internal: re-queryβ firing every ~30 seconds matchingtopic.metadata.refresh.interval.ms.leader=0responses. However, the topic-levelNO_INFOflag is never cleared. The partition scheduler checks this flag first and refuses to route messages regardless.message.timeout.msand are dropped asMessageTimedOut. Vector logsEvents dropped. No re-queue to disk buffer occurs./healthreturns 200, sources keep accepting events, disk buffer keeps growing). The failure is completely silent.What I expected:
After the broker recovers and metadata re-queries succeed with valid leader information:
NO_INFOstate and resume producing normally β consistent with how it recovers from other transient leader unavailability errors (e.g.LEADER_NOT_AVAILABLE).This bug is distinguishable from a plain
MessageTimedOuttimeout issue. The stuck state persists for the entire remaining lifetime of the Vector process (confirmed across 7+ minutes in one incident). A pod restart fully recovers it.Configuration
Kafka: Strimzi-managed KRaft cluster, single broker (
broker.session.timeout.msdefault 9000ms), and single controller. KRaft controller configured with 256m heap (GC pauses exceed 9s, triggering broker fencing).Version
vector 0.55.0 (aarch64-unknown-linux-gnu cf8de83 2026-04-22 14:31:18.008404048)
Debug Output
The following log lines capture the full failure sequence. Collected with
VECTOR_LOG=traceandlibrdkafka_options: debug: "all", log_level: "7".The trigger event (fires once, then immediately suppressed by Vector's tracing-limit):
librdkafka IS re-querying (~30s interval =
topic.metadata.refresh.interval.ms), but the topic stays stuck:Topic-level NO_INFO persists indefinitely after re-queries succeed (rate-limited to 1 per 10s by Vector's tracing-limit; underlying frequency is higher):
Messages being dropped continuously:
Individual per-partition timeouts in the internal broker thread:
Kafka broker side β confirmed the broker recovered before Vector's stuck state began:
The broker was in the fenced (
leader=-1) state for less than 1 second. Vector's NOINFO event landed at05:47:16.229β within the sub-second fencing window.Example Data
N/A β this is a connection/metadata state failure, not a data encoding issue. The failure occurs with any payload.
Additional Context
restartPolicy: Always).MessageTimedOut, they are gone./healthendpoint returns200 OKfor the entire duration of the stuck state. The standardhttpGetliveness probe does not detect this failure. A metric-based probe onvector_component_sent_events_total{component_id="kafka_logs"}would be required to auto-restart on this failure.WARNlog fornew leader-1 not found in cachefires only twice (first occurrence + "being suppressed") before Vector'stracing-limitlayer silences it for the 10-second window. Since no furtherWARN-level librdkafka events occur after the window expires, the rate-limit summary (has been suppressed N times) is never printed. From the operator's perspective the logs appear to show a brief transient error that resolved β while in reality the sink is permanently broken.02:58:18,03:16:41,05:47:16). In one incident the failure persisted for 7+ minutes until the pod was manually restarted.leader=-1path. NormalLEADER_NOT_AVAILABLEresponses (where librdkafka finds a validleader=Nbroker) appear to recover correctly.How to reproduce the
leader=-1scenario:The trigger is a KRaft controller JVM GC pause that exceeds
broker.session.timeout.ms. We reproduce it reliably using the following setup:Environment:
kraft: {}mode), single broker + single controller pool-Xms128m -Xmx128m(128m is reliable; 256m triggers it less frequently)broker.session.timeout.msleft at the default (9000ms) β no tuning needed at this size heapTrigger (two options):
Option A β natural (wait for GC): With a 128m controller heap and a modest produce workload running, the controller heap exhausts and triggers stop-the-world GC pauses of 10β30 seconds within minutes. No manual intervention needed β just watch the controller logs for
EventPerformanceMonitor: Exceptionally slow controller event ... took NNNN ms.Option B β forced (on-demand): Trigger a full GC on the controller JVM manually:
Run either command while Vector is actively producing to Kafka. The controller pause causes the broker heartbeat to time out β controller fences broker 0 β all partition leaders become -1 β broker immediately recovers via ELR. If Vector's 30s metadata refresh cycle lands in the brief fenced window, the bug triggers.
Confirmation the bug triggered (check Vector logs):
After this line, all produce to that topic will silently fail until Vector is restarted.
References
new leader-1 not found in cachemessage originates in theMETADATAUPDATEsubsystem. The suspected code path:rd_kafka_broker_find_by_nodeid(-1)returns NULL β partition assigned to internal broker sentinel β topic-levelNO_INFOset β subsequent valid metadata responses processed byrd_kafka_topic_metadata_update()update per-partition routing but do not clear the topic-levelNO_INFOflag. https://github.com/confluentinc/librdkafka/blob/01545ef06ec297c4dcf49b6821af827470d4d22b/src/rdkafka_metadata.c#L2164