Vitalina Horyukova created KAFKA-7583:
-----------------------------------------

             Summary: Producer force shutdown doesn't work when all brokers is 
down.
                 Key: KAFKA-7583
                 URL: https://issues.apache.org/jira/browse/KAFKA-7583
             Project: Kafka
          Issue Type: Bug
          Components: producer 
    Affects Versions: 0.11.0.0
         Environment: {code}
uname -a
Linux kassa 3.13.0-139-generic #188-Ubuntu SMP Tue Jan 9 14:43:09 UTC 2018 
x86_64 x86_64 x86_64 GNU/Linux
java --version
java 1.8.0_152
{code}
            Reporter: Vitalina Horyukova


Hi!
When all Kafka brokers are down, thread which called {{KafkaProducer.close}} 
the infinity stucks in second join to {{KafkaProducer.ioThread}}, because 
{{KafkaProducer.ioThread}} infinity spins over {{while}} cycle in 
{{Sender.maybeWaitForProducerId}}. The root cause of this is that 
{{Sender.awaitLeastLoadedNodeReady}} -> {{NetworkClientUtils.awaitReady}} 
throws {{IOException}} every iteration.
In logs you can see infinity repeation of this part every `retry.backoff.ms`:
{code:java}
[2018-11-01T16:19:47.583+03:00] TRACE [kafka-producer-network-thread | 
producer-1] [] [] [] [NetworkClient] Found least loaded node kafka:9093 (id: -1 
rack: null)
[2018-11-01T16:19:47.583+03:00] TRACE [kafka-producer-network-thread | 
producer-1] [] [] [] [NetworkClient] Found least loaded node kafka:9093 (id: -1 
rack: null)
[2018-11-01T16:19:47.583+03:00] DEBUG [kafka-producer-network-thread | 
producer-1] [] [] [] [NetworkClient] Initialize connection to node -1 for 
sending metadata request
[2018-11-01T16:19:47.583+03:00] DEBUG [kafka-producer-network-thread | 
producer-1] [] [] [] [NetworkClient] Initiating connection to node -1 at 
kafka:9093.
[2018-11-01T16:19:47.585+03:00] DEBUG [kafka-producer-network-thread | 
producer-1] [] [] [] [Selector] Connection with kafka/xxx.xxx.xxx.xxx 
disconnected
java.net.ConnectException: Connection refused
        at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
        at 
sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
        at 
org.apache.kafka.common.network.SslTransportLayer.finishConnect(SslTransportLayer.java:109)
        at 
org.apache.kafka.common.network.KafkaChannel.finishConnect(KafkaChannel.java:95)
        at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:359)
        at org.apache.kafka.common.network.Selector.poll(Selector.java:326)
        at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:432)
        at 
org.apache.kafka.clients.NetworkClientUtils.isReady(NetworkClientUtils.java:39)
        at 
org.apache.kafka.clients.NetworkClientUtils.awaitReady(NetworkClientUtils.java:62)
        at 
org.apache.kafka.clients.producer.internals.Sender.awaitLeastLoadedNodeReady(Sender.java:409)
        at 
org.apache.kafka.clients.producer.internals.Sender.maybeWaitForProducerId(Sender.java:418)
        at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:203)
        at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:162)
        at java.lang.Thread.run(Thread.java:748)
[2018-11-01T16:19:47.585+03:00] DEBUG [kafka-producer-network-thread | 
producer-1] [] [] [] [NetworkClient] Node -1 disconnected.
[2018-11-01T16:19:47.585+03:00] WARN  [kafka-producer-network-thread | 
producer-1] [] [] [] [NetworkClient] Connection to node -1 could not be 
established. Broker may not be available.
[2018-11-01T16:19:47.585+03:00] DEBUG [kafka-producer-network-thread | 
producer-1] [] [] [] [Sender] Broker {} disconnected while awaiting 
InitProducerId response
java.io.IOException: Connection to kafka:9093 (id: -1 rack: null) failed.
        at 
org.apache.kafka.clients.NetworkClientUtils.awaitReady(NetworkClientUtils.java:68)
        at 
org.apache.kafka.clients.producer.internals.Sender.awaitLeastLoadedNodeReady(Sender.java:409)
        at 
org.apache.kafka.clients.producer.internals.Sender.maybeWaitForProducerId(Sender.java:418)
        at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:203)
        at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:162)
        at java.lang.Thread.run(Thread.java:748)
[2018-11-01T16:19:47.585+03:00] TRACE [kafka-producer-network-thread | 
producer-1] [] [] [] [Sender] Retry InitProducerIdRequest in 100ms.
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to