librdkafka
librdkafka copied to clipboard
Infinity timeout does not work with message.timeout.ms=0
Read the FAQ first: https://github.com/edenhill/librdkafka/wiki/FAQ
Description
The library must treat it as infinity timeout.
The problem:
1. rd_kafka_ProduceRequest converts INT_MAX64 to int and get a negative value
2. than multiply on 1000 and overflow integer one more time and get a positive value
The behavior is invalid but still sometimes get a false-positive result
How to reproduce
- Set
message.timeout.ms=0andrequest.required.acks=-1. Timeout will - Wait monotonic clock reach
17179869075 - Enjoi all your requests timeout in several milliseconds
IMPORTANT: Always try to reproduce the issue on the latest released version (see https://github.com/edenhill/librdkafka/releases), if it can't be reproduced on the latest version the issue has been fixed.
Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
- [x] librdkafka version (release number or git tag):
<REPLACE with e.g., v0.10.5 or a git sha. NOT "latest" or "current"> - [ ] Apache Kafka version:
<REPLACE with e.g., 0.10.2.3> - [ ] librdkafka client configuration:
<REPLACE with e.g., message.timeout.ms=123, auto.reset.offset=earliest, ..> - [ ] Operating system:
<REPLACE with e.g., Centos 5 (x64)> - [ ] Provide logs (with
debug=..as necessary) from librdkafka - [ ] Provide broker log excerpts
- [ ] Critical issue
Thank you for your analysis, will look into it.
Hi, @edenhill ! we have found the same bug: msg may be timed out even if message.timeout.ts=0.
Could you say if there is any way to bypass this bug in v0.11.5 version ?
May be the following information will help you in reproducing:
- Source code and logs are available via link: https://gist.github.com/KseniyaYakil/8b316dc80bdd60011286aae31054a74b
- librdkafka versions: tested in v0.11.5 and master(849c066b5599)
- librdkafka configuration:
broker_version_fallback="0.10.1";
queue_buffering_max_messages="10000";
queue_buffering_max_ms="100";
queue_buffering_max_kbytes="1269532";
batch_num_messages="10000";
max_in_flight_requests_per_connection="1";
message_send_max_retries="3600";
retry_backoff_ms="30000";
compression_codec="snappy";
socket_timeout_ms="300000";
socket_max_fails="1";
socket_keepalive_enable="true";
topic_metadata_refresh_interval_ms="180000";
request_required_acks="1";
request_timeout_ms="12000";
message_timeout_ms="0";
log_connection_close="false";
log_level="6";
statistics_interval_ms="10000";
debug="all"; - How to reproduce 4.1) build prog gcc rdkafka_queue_producer.c /usr/local/lib/librdkafka.a -ldl -lz -lpthread -lrt -lm -o rdkafka_queue_test -ggdb3 4.2) run ./rdkafka_queue_test kafka1.d3:9092,kafka2.d3:9092 kseniya_rdkafka_test 1 2> rdkafka.log 4.3) Don’t press any key, open another terminal window and see in rdkafka.log the following last msg in log Thu Oct 11 19:43:05 2018 : 7: METADATA: [thrd:main]: kafka1.d3:9092/1: 1/1 requested topic(s) seen in metadata Then drop all packets to all kafka brokers (for example via iptables) iptables -t filter -A OUTPUT -4 -p tcp -d kafka1.d3 -j DROP iptables -t filter -A OUTPUT -4 -p tcp -d kafka2.d3 -j DROP 4.4) return to the first terminal to main prog and press any key 4.5) see in log : 7: REQTMOUT: [thrd:kafka1.d3:9092/bootstrap]: kafka1.d3:9092/1: Timed out 1+0+0 requests Thu Oct 11 19:48:24 2018 : 7: BROKERFAIL: [thrd:kafka1.d3:9092/bootstrap]: kafka1.d3:9092/1: failed: err: Local: Timed out: (errno: Connection timed out) ... recieved err: -185: kafka1.d3:9092/1: 1 request(s) timed out: disconnect 4.6) notice that time diff between “iptables” and “Timed out” is about 1 second And all original timeout configuration for message are bigger than 10 seconds
Can't you simply set the limit to rkm->rkm_ts_timeout = INT64_MAX - now; in case of 0? That would avoid the negative value. You still have to fix the linger.ms case.