Graham Campbell created KAFKA-16902:
---------------------------------------
Summary: Sender ignores socket connection timeout when
reconnecting to transaction coordinator
Key: KAFKA-16902
URL: https://issues.apache.org/jira/browse/KAFKA-16902
Project: Kafka
Issue Type: Bug
Components: producer
Affects Versions: 3.6.2, 3.7.0, 3.5.2, 3.4.1, 3.3.2, 3.2.3, 3.1.2, 2.8.2
Reporter: Graham Campbell
maybeSendAndPollTransactionalRequest checks that the required coordinator node
is ready before sending transactional requests. It uses
{{Sender::awaitNodeReady}} and {{NetworkClientUtils::awaitReady}} to do so in a
blocking manner.
If the NetworkClient is in the middle of reconnecting to the coordinator due to
a broker side disconnection (eg. broker restart), the
socket.connection.setup.timeout.ms config (default 10 seconds) is ignored and
request.timeout.ms is used (35 seconds). This results in up to 25 extra seconds
of waiting before hitting the expected timeout. In my connectivity-related
exceptions are sometimes thrown before the full 35 second timeout (eg.
NoRouteToHostException, UnknownHostException).
[awaitNodeReady uses
requestTimeoutMs|https://github.com/apache/kafka/blob/896af1b2f2f2a7d579e0aef074bcb2004c0246f2/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java#L568]
as the timeout, which [awaitReady passes to the NetworkClient as the poll
timeout|https://github.com/apache/kafka/blob/896af1b2f2f2a7d579e0aef074bcb2004c0246f2/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java#L73-L74],
so disconnections aren't processed until after the [selector.poll
completes|https://github.com/apache/kafka/blob/896af1b2f2f2a7d579e0aef074bcb2004c0246f2/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java#L585].
Logs from a local test with request.timeout.ms=15000 and a patched client with
some extra logging show the timeout being violated:
{code:java}
2024-05-29 21:31:49:736 +0000 [kafka-producer-network-thread | t7] INFO
org.apache.kafka.clients.NetworkClient - [Producer clientId=t7,
transactionalId=MyT7] Disconnecting from node 11 due to socket connection setup
timeout. The timeout value is 10351 ms. 15013 ms since last attempt
2024-05-29 21:31:49:736 +0000 [kafka-producer-network-thread | t7] INFO
org.apache.kafka.clients.producer.internals.Sender - [Producer clientId=t7,
transactionalId=MyT7] awaitReady timed out after 15013 ms (threshold of 15000
ms)
2024-05-29 21:31:49:752 +0000 [kafka-producer-network-thread | t7] INFO
org.apache.kafka.clients.producer.internals.TransactionManager - [Producer
clientId=t7, transactionalId=MyT7] Discovered transaction coordinator
kafka-2.redacted:9093 (id: 2 rack: null) {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)