Mahdi created KAFKA-2864:
----------------------------

             Summary: Bad zookeeper host causes broker to shutdown uncleanly 
and stall producers
                 Key: KAFKA-2864
                 URL: https://issues.apache.org/jira/browse/KAFKA-2864
             Project: Kafka
          Issue Type: Bug
          Components: zkclient
    Affects Versions: 0.8.2.1
            Reporter: Mahdi
            Priority: Critical


We are using kafka 0.8.2.1 and we noticed that kafka/zookeeper-client were not 
able to gracefully handle a non existing zookeeper instance. This caused one of 
our brokers to get stuck during a self-inflicted shutdown and that seemed to 
impact the partitions for which the broker was a leader even though we had two 
other replicas.

Here is a timeline of what happened (shortened for brevity, I'll attach log 
snippets):

We have a 7 node zookeeper cluster. Two of our nodes were decommissioned and 
their dns records removed (zookeeper15 and zookeeper16). The decommissioning 
happened about two weeks earlier. We noticed the following in the logs

- Opening socket connection to server ip-10-0-0-1.ec2.internal/10.0.0.1:2181. 
Will not attempt to authenticate using SASL (unknown error)
- Client session timed out, have not heard from server in 858ms for sessionid 
0x1250c5c0f1f5001c, closing socket connection and attempting reconnect
- Opening socket connection to server ip-10.0.0.2.ec2.internal/10.0.0.2:2181. 
Will not attempt to authenticate using SASL (unknown error)
- zookeeper state changed (Disconnected)
- Client session timed out, have not heard from server in 2677ms for sessionid 
0x1250c5c0f1f5001c, closing socket connection and attempting reconnect
- Opening socket connection to server ip-10.0.0.3.ec2.internal/10.0.0.3:2181. 
Will not attempt to authenticate using SASL (unknown error)
- Socket connection established to ip-10.0.0.3.ec2.internal/10.0.0.3:2181, 
initiating session
- zookeeper state changed (Expired)
- Initiating client connection, 
connectString=zookeeper21.example.com:2181,zookeeper19.example.com:2181,zookeeper22.example.com:2181,zookeeper18.example.com:2181,zookeeper20.example.com:2181,zookeeper16.example.com:2181,zookeeper15.example.com:2181/foo/kafka/central
 sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@3bbc39f8
- Unable to reconnect to ZooKeeper service, session 0x1250c5c0f1f5001c has 
expired, closing socket connection
- Unable to re-establish connection. Notifying consumer of the following 
exception:
org.I0Itec.zkclient.exception.ZkException: Unable to connect to 
zookeeper21.example.com:2181,zookeeper19.example.com:2181,zookeeper22.example.com:2181,zookeeper18.example.com:2181,zookeeper20.example.com:2181,zookeeper16.example.com:2181,zookeeper15.example.com:2181/foo/kafka/central
        at org.I0Itec.zkclient.ZkConnection.connect(ZkConnection.java:69)
        at org.I0Itec.zkclient.ZkClient.reconnect(ZkClient.java:1176)
        at org.I0Itec.zkclient.ZkClient.processStateChanged(ZkClient.java:649)
        at org.I0Itec.zkclient.ZkClient.process(ZkClient.java:560)
        at 
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:522)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
Caused by: java.net.UnknownHostException: zookeeper16.example.com: unknown error
        at java.net.Inet6AddressImpl.lookupAllHostAddr(Native Method)
        at java.net.InetAddress$2.lookupAllHostAddr(InetAddress.java:928)
        at 
java.net.InetAddress.getAddressesFromNameService(InetAddress.java:1323)
        at java.net.InetAddress.getAllByName0(InetAddress.java:1276)
        at java.net.InetAddress.getAllByName(InetAddress.java:1192)
        at java.net.InetAddress.getAllByName(InetAddress.java:1126)
        at 
org.apache.zookeeper.client.StaticHostProvider.<init>(StaticHostProvider.java:61)
        at org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:445)
        at org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:380)
        at org.I0Itec.zkclient.ZkConnection.connect(ZkConnection.java:67)
        ... 5 more


That seems to have caused the following:
 [main-EventThread] [org.apache.zookeeper.ClientCnxn     ]: EventThread shut 
down

Which in turn caused kafka to shut itself down
[Thread-2] [kafka.server.KafkaServer            ]: [Kafka Server 13], shutting 
down
[Thread-2] [kafka.server.KafkaServer            ]: [Kafka Server 13], Starting 
controlled shutdown

However, the shutdown didn't go as expected apparently due to an NPE in the zk 
client

2015-11-12T12:03:40.101Z WARN  [Thread-2                           ] 
[kafka.utils.Utils$                  ]:
java.lang.NullPointerException
        at org.I0Itec.zkclient.ZkConnection.readData(ZkConnection.java:117)
        at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:992)
        at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:988)
        at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:883)
        at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:988)
        at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:983)
        at kafka.utils.ZkUtils$.readDataMaybeNull(ZkUtils.scala:450)
        at kafka.utils.ZkUtils$.getController(ZkUtils.scala:65)
        at 
kafka.server.KafkaServer.kafka$server$KafkaServer$$controlledShutdown(KafkaServer.scala:194)
        at 
kafka.server.KafkaServer$$anonfun$shutdown$1.apply$mcV$sp(KafkaServer.scala:269)
        at kafka.utils.Utils$.swallow(Utils.scala:172)
        at kafka.utils.Logging$class.swallowWarn(Logging.scala:92)
        at kafka.utils.Utils$.swallowWarn(Utils.scala:45)
        at kafka.utils.Logging$class.swallow(Logging.scala:94)
        at kafka.utils.Utils$.swallow(Utils.scala:45)
        at kafka.server.KafkaServer.shutdown(KafkaServer.scala:269)
        at 
kafka.server.KafkaServerStartable.shutdown(KafkaServerStartable.scala:42)
        at kafka.Kafka$$anon$1.run(Kafka.scala:42)
2015-11-12T12:03:40.106Z INFO  [Thread-2                           ] 
[kafka.network.SocketServer          ]: [Socket Server on Broker 13], Shutting 
down

The kafka process continued running after this point. This is confirmed by the 
continuous rolling of logs
[ReplicaFetcherThread-3-9           ] [kafka.log.Log                       ]: 
Rolled new log segment for 'topic-a-1' in 0 ms.
[ReplicaFetcherThread-0-12          ] [kafka.log.Log                       ]: 
Rolled new log segment for 'topic-b-4' in 0 ms.

etc..

At this point, that broker was in a half-dead state. Our clients were still 
timing out enqueuing messages to it. The under-replicated partition count on 
the other brokers was stuck at a positive, constant value and did not make any 
progress. We also noticed that the jmx connector threads weren't responding, 
which is how we found out that the process was in a bad shape. This happened 
for about 40mn till we killed the process and restarted it. Things have 
recovered after the restart.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to