Skip to content

Commit

Permalink
Trigger broker connection-retry (when down) if any requests are enqueued
Browse files Browse the repository at this point in the history
  • Loading branch information
edenhill committed Apr 2, 2020
1 parent 1e8e6e0 commit e4a8c0f
Showing 1 changed file with 5 additions and 2 deletions.
7 changes: 5 additions & 2 deletions src/rdkafka_broker.c
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down

0 comments on commit e4a8c0f

Please sign in to comment.