Haruki Okada created KAFKA-14445: ------------------------------------ Summary: Producer doesn't request metadata update on REQUEST_TIMED_OUT Key: KAFKA-14445 URL: https://issues.apache.org/jira/browse/KAFKA-14445 Project: Kafka Issue Type: Improvement Reporter: Haruki Okada
Produce requests may fail with timeout by `request.timeout.ms` in below two cases: * Didn't receive produce response within `request.timeout.ms` * Produce response received, but it ended up with `REQUEST_TIMEOUT_MS` in the broker Former case usually happens when a broker-machine got failed or there's network glitch etc. In this case, the connection will be disconnected and metadata-update will be requested to discover new leader: [https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java#L556] The problem is in latter case (REQUEST_TIMED_OUT on the broker). In this case, the produce request will be ended up with TimeoutException, which doesn't inherit InvalidMetadataException so it doesn't trigger metadata update. Typical cause of REQUEST_TIMED_OUT is replication delay due to follower-side problem, that metadata-update doesn't make much sense indeed. However, we found that in some cases, stale metadata on REQUEST_TIMED_OUT could cause produce requests to retry unnecessarily , which may end up with batch expiration due to delivery timeout. Below is the scenario we experienced: * Environment: ** Partition tp-0 has 3 replicas, 1, 2, 3. Leader is 1 ** min.insync.replicas=2 ** acks=all * Scenario: ** broker 1 "partially" failed *** It lost ZooKeeper connection and kicked out from the cluster **** There was controller log like: ***** {code:java} [2022-12-04 08:01:04,013] INFO [Controller id=XX] Newly added brokers: , deleted brokers: 1, bounced brokers: {code} *** However, somehow the broker was able continued to receive produce requests **** We're still working on investigating how this is possible though. **** Indeed, broker 1 was somewhat "alive" and keeps working according to server.log *** In other words, broker 1 became "zombie" ** broker 2 was elected as new leader *** broker 3 became follower of broker 2 *** However, since broker 1 was still out of cluster, it didn't receive LeaderAndIsr so 1 kept thinking itself as the leader of tp-0 ** Meanwhile, producer keeps sending produce requests to broker 1 and requests were failed due to REQUEST_TIMED_OUT because no brokers replicates from broker 1. *** REQUEST_TIMED_OUT doesn't trigger metadata update, so produce didn't have a change to update its stale metadata So I suggest to request metadata update even on REQUEST_TIMED_OUT exception, for the case that the old leader became "zombie" -- This message was sent by Atlassian Jira (v8.20.10#820010)