From e4a8c0f62742789bd4d62accd1497c82c08c4259 Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Thu, 2 Apr 2020 17:22:30 +0200 Subject: [PATCH] Trigger broker connection-retry (when down) if any requests are enqueued --- src/rdkafka_broker.c | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/rdkafka_broker.c b/src/rdkafka_broker.c index 9235cb2587..0c4d992957 100644 --- a/src/rdkafka_broker.c +++ b/src/rdkafka_broker.c @@ -4794,8 +4794,11 @@ static void rd_kafka_broker_serve (rd_kafka_broker_t *rkb, int timeout_ms) { /* rkb_persistconn.internal is the per broker_serve() * automatic counter that keeps track of anything * in the producer/consumer logic needs this broker connection - * to be up. */ - rkb->rkb_persistconn.internal = 0; + * to be up. + * The value is reset here on each serve(). If there are queued + * requests we know right away that a connection is needed. */ + rkb->rkb_persistconn.internal = + rd_atomic32_get(&rkb->rkb_outbufs.rkbq_cnt) > 0; if (rkb->rkb_source == RD_KAFKA_INTERNAL) rd_kafka_broker_internal_serve(rkb, abs_timeout);