Skip to content

Commit

Permalink
Fix timeout for rd_kafka_query_watermark_offsets (#4460)
Browse files Browse the repository at this point in the history
  • Loading branch information
milindl authored Oct 11, 2023
1 parent f64242d commit 35df9d6
Show file tree
Hide file tree
Showing 4 changed files with 64 additions and 7 deletions.
8 changes: 6 additions & 2 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ librdkafka v2.3.0 is a feature release:
* [KIP-430](https://cwiki.apache.org/confluence/display/KAFKA/KIP-430+-+Return+Authorized+Operations+in+Describe+Responses):
Return authorized operations in Describe Responses.
(#4240, @jainruchir).
* [KIP-580](https://cwiki.apache.org/confluence/display/KAFKA/KIP-580%3A+Exponential+Backoff+for+Kafka+Clients): Added Exponential Backoff mechanism for
* [KIP-580](https://cwiki.apache.org/confluence/display/KAFKA/KIP-580%3A+Exponential+Backoff+for+Kafka+Clients): Added Exponential Backoff mechanism for
retriable requests with `retry.backoff.ms` as minimum backoff and `retry.backoff.max.ms` as the
maximum backoff, with 20% jitter(#4422).
* Fixed ListConsumerGroupOffsets not fetching offsets for all the topics in a group with Apache Kafka version below 2.4.0.
Expand All @@ -34,6 +34,7 @@ librdkafka v2.3.0 is a feature release:
* Fix to ensure max.poll.interval.ms is reset when rd_kafka_poll is called with
consume_cb (#4431).
* Fix for idempotent producer fatal errors, triggered after a possibly persisted message state (#4438).
* Fix `rd_kafka_query_watermark_offsets` continuing beyond timeout expiry (#4460).


## Upgrade considerations
Expand All @@ -42,7 +43,7 @@ librdkafka v2.3.0 is a feature release:
If it is set greater than `retry.backoff.max.ms` which has the default value of 1000 ms then it is assumes the value of `retry.backoff.max.ms`.
To change this behaviour make sure that `retry.backoff.ms` is always less than `retry.backoff.max.ms`.
If equal then the backoff will be linear instead of exponential.

* `topic.metadata.refresh.fast.interval.ms`:
If it is set greater than `retry.backoff.max.ms` which has the default value of 1000 ms then it is assumes the value of `retry.backoff.max.ms`.
To change this behaviour make sure that `topic.metadata.refresh.fast.interval.ms` is always less than `retry.backoff.max.ms`.
Expand All @@ -56,6 +57,9 @@ librdkafka v2.3.0 is a feature release:
* An assertion failed with insufficient buffer size when allocating
rack information on 32bit architectures.
Solved by aligning all allocations to the maximum allowed word size (#4449).
* The timeout for `rd_kafka_query_watermark_offsets` was not checked after
making the necessary ListOffsets requests, and thus, it never timed out in
case of broker/network issues. Fixed by checking timeout expiry (#4460).

### Idempotent producer fixes

Expand Down
15 changes: 10 additions & 5 deletions src/rdkafka.c
Original file line number Diff line number Diff line change
Expand Up @@ -3544,6 +3544,7 @@ rd_kafka_resp_err_t rd_kafka_query_watermark_offsets(rd_kafka_t *rk,
struct rd_kafka_partition_leader *leader;
rd_list_t leaders;
rd_kafka_resp_err_t err;
int tmout;

partitions = rd_kafka_topic_partition_list_new(1);
rktpar =
Expand Down Expand Up @@ -3590,11 +3591,15 @@ rd_kafka_resp_err_t rd_kafka_query_watermark_offsets(rd_kafka_t *rk,
rd_list_destroy(&leaders);

/* Wait for reply (or timeout) */
while (state.err == RD_KAFKA_RESP_ERR__IN_PROGRESS &&
rd_kafka_q_serve(rkq, 100, 0, RD_KAFKA_Q_CB_CALLBACK,
rd_kafka_poll_cb,
NULL) != RD_KAFKA_OP_RES_YIELD)
;
while (state.err == RD_KAFKA_RESP_ERR__IN_PROGRESS) {
tmout = rd_timeout_remains(ts_end);
if (rd_timeout_expired(tmout)) {
state.err = RD_KAFKA_RESP_ERR__TIMED_OUT;
break;
}
rd_kafka_q_serve(rkq, tmout, 0, RD_KAFKA_Q_CB_CALLBACK,
rd_kafka_poll_cb, NULL);
}

rd_kafka_q_destroy_owner(rkq);

Expand Down
46 changes: 46 additions & 0 deletions tests/0031-get_offsets.c
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
/* Typical include path would be <librdkafka/rdkafka.h>, but this program
* is built from within the librdkafka source tree and thus differs. */
#include "rdkafka.h" /* for Kafka driver */
#include "../src/rdkafka_proto.h"


/**
Expand Down Expand Up @@ -117,3 +118,48 @@ int main_0031_get_offsets(int argc, char **argv) {

return 0;
}

/*
* Verify that rd_kafka_query_watermark_offsets times out in case we're unable
* to fetch offsets within the timeout (Issue #2588).
*/
int main_0031_get_offsets_mock(int argc, char **argv) {
int64_t qry_low, qry_high;
rd_kafka_resp_err_t err;
const char *topic = test_mk_topic_name(__FUNCTION__, 1);
rd_kafka_mock_cluster_t *mcluster;
rd_kafka_t *rk;
rd_kafka_conf_t *conf;
const char *bootstraps;
const int timeout_ms = 1000;

if (test_needs_auth()) {
TEST_SKIP("Mock cluster does not support SSL/SASL\n");
return 0;
}

mcluster = test_mock_cluster_new(1, &bootstraps);
rd_kafka_mock_topic_create(mcluster, topic, 1, 1);
rd_kafka_mock_broker_push_request_error_rtts(
mcluster, 1, RD_KAFKAP_ListOffsets, 1, RD_KAFKA_RESP_ERR_NO_ERROR,
(int)(timeout_ms * 1.2));

test_conf_init(&conf, NULL, 30);
test_conf_set(conf, "bootstrap.servers", bootstraps);
rk = test_create_handle(RD_KAFKA_PRODUCER, conf);


err = rd_kafka_query_watermark_offsets(rk, topic, 0, &qry_low,
&qry_high, timeout_ms);

TEST_ASSERT(err == RD_KAFKA_RESP_ERR__TIMED_OUT,
"Querying watermark offsets should fail with %s when RTT > "
"timeout, instead got %s",
rd_kafka_err2name(RD_KAFKA_RESP_ERR__TIMED_OUT),
rd_kafka_err2name(err));

rd_kafka_destroy(rk);
test_mock_cluster_destroy(mcluster);

return 0;
}
2 changes: 2 additions & 0 deletions tests/test.c
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,7 @@ _TEST_DECL(0028_long_topicnames);
_TEST_DECL(0029_assign_offset);
_TEST_DECL(0030_offset_commit);
_TEST_DECL(0031_get_offsets);
_TEST_DECL(0031_get_offsets_mock);
_TEST_DECL(0033_regex_subscribe);
_TEST_DECL(0033_regex_subscribe_local);
_TEST_DECL(0034_offset_reset);
Expand Down Expand Up @@ -334,6 +335,7 @@ struct test tests[] = {
/* Loops over committed() until timeout */
_THRES(.ucpu = 10.0, .scpu = 5.0)),
_TEST(0031_get_offsets, 0),
_TEST(0031_get_offsets_mock, TEST_F_LOCAL),
_TEST(0033_regex_subscribe, 0, TEST_BRKVER(0, 9, 0, 0)),
_TEST(0033_regex_subscribe_local, TEST_F_LOCAL),
_TEST(0034_offset_reset, 0),
Expand Down

0 comments on commit 35df9d6

Please sign in to comment.