Skip to content

Commit

Permalink
Prefer least-idle connection when randomly selectiong broker for meta…
Browse files Browse the repository at this point in the history
…data queries, etc. (#2845)

Reasons:
- this connection is most likely to function properly.
- allows truly idle connections to be killed by the broker's/LB's
  idle connection reaper.

This will fix issues like "Metadata request timed out" when the periodic
metadata refresh picks an idle (typically bootstrap) connection which
has exceeded the load-balancer's idle time (which silently kills the
connection without sending FIN/RST to the client).
  • Loading branch information
edenhill committed May 11, 2020
1 parent 0cf428b commit 0527457
Show file tree
Hide file tree
Showing 7 changed files with 187 additions and 114 deletions.
3 changes: 3 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,9 @@ librdkafka.
topics to be propagated throughout the cluster before reporting them
as non-existent. This fixes race issues where CreateTopics() is
quickly followed by produce().
* Prefer least idle connection for periodic metadata refreshes, et.al.,
to allow truly idle connections to time out and to avoid load-balancer-killed
idle connection errors (#2845)
* Added Test scenarios which define the cluster configuration
* Add MinGW-w64 builds (@ed-alertedh, #2553)
* `./configure --enable-XYZ` now requires the XYZ check to pass,
Expand Down
6 changes: 3 additions & 3 deletions src/rdkafka.c
Original file line number Diff line number Diff line change
Expand Up @@ -1816,9 +1816,9 @@ static void rd_kafka_metadata_refresh_cb (rd_kafka_timers_t *rkts, void *arg) {
rd_kafka_t *rk = rkts->rkts_rk;
int sparse = 1;

/* Dont do sparse requests if there is a consumer group with an
* active subscription since subscriptions need to be able to match
* on all topics. */
/* Dont do sparse requests if there is a consumer group with an active
* wildcard subscription since it needs to be able to match on all
* topics. */
if (rk->rk_type == RD_KAFKA_CONSUMER && rk->rk_cgrp &&
rk->rk_cgrp->rkcg_flags & RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION)
sparse = 0;
Expand Down
Loading

0 comments on commit 0527457

Please sign in to comment.