You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
When rd_kafka_query_watermark_offsets is called with a timeout below 1000ms it will most likely not timeout within that time.
rd_kafka_query_watermark_offsets sends one request to get the low watermark, and one request to get the high watermark. These requests are then put in a queue, which is checked every second to see if any request has timed out.
If a response arrives before this check, indifferent to the request timeout, then rd_kafka_query_watermark_offsets returns the received offsets.
#4460 contained a fix for the timeout, but was replaced with #4225 which gives the inaccurate timeout handling.
> ./interactive_broker_version.py 3.4.0
> TESTS=0031 make run_seq
...
[0031_get_offsets_mock / 0.002s] Created kafka instance 0031_get_offsets_mock#producer-4
[0031_get_offsets_mock / 0.166s] TEST FAILURE
### Test "0031_get_offsets_mock (test_query_watermark_offsets_timeout:55)" failed at 0031-get_offsets.c:71:test_query_watermark_offsets_timeout() at Fri Oct 18 09:25:59 2024: ###
Test assertion failed: "err == RD_KAFKA_RESP_ERR__TIMED_OUT": Querying watermark offsets should fail with _TIMED_OUT when RTT > timeout, instead got NO_ERROR
which gives a NO_ERROR, i.e. no timeout occurred in rd_kafka_query_watermark_offsets.
Note that the testcase took 0.166s. The issue has also been verified by "printf-debugging".
Optionally: update the mock to respond after 800ms, instead of (int)(timeout_ms * 1.2)), the call will still not timeout after 100ms.
[0031_get_offsets_mock / 0.003s] Created kafka instance 0031_get_offsets_mock#producer-4
[0031_get_offsets_mock / 0.847s] TEST FAILURE
### Test "0031_get_offsets_mock (test_query_watermark_offsets_timeout:55)" failed at 0031-get_offsets.c:71:test_query_watermark_offsets_timeout() at Fri Oct 18 10:16:46 2024: ###
Test assertion failed: "err == RD_KAFKA_RESP_ERR__TIMED_OUT": Querying watermark offsets should fail with _TIMED_OUT when RTT > timeout, instead got NO_ERROR
Checklist
librdkafka version (release number or git tag): v2.6.0 and master
Apache Kafka version: 3.4.0
librdkafka client configuration: as used in our tests.
Operating system: any
Provide logs (with debug=.. as necessary) from librdkafka
Provide broker log excerpts
Critical issue
The text was updated successfully, but these errors were encountered:
Description
When
rd_kafka_query_watermark_offsets
is called with a timeout below 1000ms it will most likely not timeout within that time.rd_kafka_query_watermark_offsets
sends one request to get the low watermark, and one request to get the high watermark. These requests are then put in a queue, which is checked every second to see if any request has timed out.If a response arrives before this check, indifferent to the request timeout, then
rd_kafka_query_watermark_offsets
returns the received offsets.#4460 contained a fix for the timeout, but was replaced with #4225 which gives the inaccurate timeout handling.
Proposed correction
Use a timeout when calling
rd_kafka_q_serve
.This is how its solved in other APIs like rd_kafka_list_groups and rd_kafka_offsets_for_times, and also used in #4460.
How to reproduce
Lower the timeout used in the testcase to a value under 1s, like 100ms.
Run testcase:
which gives a
NO_ERROR
, i.e. no timeout occurred inrd_kafka_query_watermark_offsets
.Note that the testcase took 0.166s. The issue has also been verified by "printf-debugging".
Optionally: update the mock to respond after 800ms, instead of
(int)(timeout_ms * 1.2))
, the call will still not timeout after 100ms.Checklist
v2.6.0
andmaster
3.4.0
debug=..
as necessary) from librdkafkaThe text was updated successfully, but these errors were encountered: