Re: ZK connection broken to one server shutting down entire quorum
in ZOOKEEPER-3769 and ZOOKEEPER-3016 the following type of log lines were showing the problem: 03/24/20 11:16:16,297 [WorkerReceiver[myid=1]] ERROR [org.apache.zookeeper.server.NIOServerCnxnFactory] (NIOServerCnxnFactory.java:92) - Thread Thread[WorkerReceiver[myid=1],5,main] died But I am not sure about the logs on 3.4.14. Also there are multiple threads that can die here, not only the WorkerReceiver. But the death of the thread will always be logged by the NIOServerCnxnFactory. Maybe try to grep on "NIOServerCnxnFactory" and on "died". If you find anything, then look for errors / exceptions around this log line to see what happened. On Wed, Apr 22, 2020 at 6:01 PM blb.dev wrote: > Hi, thank you for the response! > > When you say maybe "some internal listener thread in the leader (zoo3) > died" > is there a particular string I could search in the logs to look for that? > > We plan on upgrading - waiting for 3.6.1 as we've had some issues moving to > 3.6.0. > > > > -- > Sent from: http://zookeeper-user.578899.n2.nabble.com/ >
Re: ZK connection broken to one server shutting down entire quorum
Hi, thank you for the response! When you say maybe "some internal listener thread in the leader (zoo3) died" is there a particular string I could search in the logs to look for that? We plan on upgrading - waiting for 3.6.1 as we've had some issues moving to 3.6.0. -- Sent from: http://zookeeper-user.578899.n2.nabble.com/
Re: ZK connection broken to one server shutting down entire quorum
Hello! as far as I can tell, the provided logs are not enough to determine the exact root cause of the problem. Maybe someone else will have a better idea, but my best guess would be that some internal listener thread in the leader (zoo3) died before, so it was not able to parse the leader election messages from zoo1 and/or zoo2. When you restarted the leader, then the listener threads re-initialized, so everything went back to normal. There were a couple of issues like this reported already: - https://issues.apache.org/jira/browse/ZOOKEEPER-2938 - https://issues.apache.org/jira/browse/ZOOKEEPER-2186 - https://issues.apache.org/jira/browse/ZOOKEEPER-3016 ... ZooKeeper 3.4.14 should already contain the fix for these above. However, we just recently fixed a similar issue: https://issues.apache.org/jira/browse/ZOOKEEPER-3769 this fix will be part of 3.6.1 and 3.5.8. Of course, it is possible that you were hitting an independent / unknown issue... We would need all the logs to verify that (the logs from each ZooKeeper servers since their last restart before starting the rolling upgrade). Anyway, I strongly suggest to upgrade your ZooKeeper cluster, as the 3.4 will be EOL soon, see the announcement: https://mail-archives.apache.org/mod_mbox/zookeeper-user/202004.mbox/browser Kind regards, Mate On Wed, Apr 22, 2020 at 2:14 AM blb.dev wrote: > Hi team, > > During a recent patching for our ZK quorum, we experienced an unrecoverable > outage. We have performed patches like this many times previously and is > working fine in other environments of ours. The goal was to shut down each > server one by one and provide patch updates then restart. However, this > time, when zoo1 (follower) was shut down, the leader (zoo3) shutdown > connection with remaining follower (zoo2) as well. > > What would cause the entire quorum to shutdown and not recover due to > stopping only zoo1? > > Running zookeeper 3.4.14 in docker containers. > zk_version 3.4.14-4c25d480e66aadd371de8bd2fd8da255ac140bcf, built on > 03/06/2019 16:18 GMT > > > *Config for master nodes:* > tickTime=2000 > maxClientCnxns=0 > dataDir=/data > dataLogDir=/datalog > clientPort=2181 > secureClientPort=2281 > initLimit=10 > syncLimit=5 > autopurge.snapRetainCount=10 > autopurge.purgeInterval=24 > > server.1=zoo1:2888:3888 > server.2=zoo2:2888:3888 > server.3=zoo3:2888:3888 > server.4=zoo4:2888:3888:observer > server.5=zoo5:2888:3888:observer > server.6=zoo6:2888:3888:observer > > *Config for observer nodes:* > tickTime=2000 > maxClientCnxns=0 > dataDir=/data > dataLogDir=/datalog > clientPort=2181 > secureClientPort=2281 > initLimit=10 > syncLimit=5 > autopurge.snapRetainCount=10 > autopurge.purgeInterval=24 > peerType=observer > server.1=zoo1:2888:3888 > server.2=zoo2:2888:3888 > server.3=zoo3:2888:3888 > server.4=zoo4:2888:3888:observer > server.5=zoo5:2888:3888:observer > server.6=zoo6:2888:3888:observer > > zoo1-zoo6 are the FQDNs of each server. > > Shutdown of zoo1 and quorum outage at 05:02 UTC > > > *Logs on zoo3 (leader):* > 2020-04-21 05:02:00,120 [myid:3] - WARN > [RecvWorker:1:QuorumCnxManager$RecvWorker@1028] - Interrupting SendWorker > java.io.EOFException > at java.io.DataInputStream.readInt(DataInputStream.java:392) > at > > org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:1010) > 2020-04-21 05:02:00,096 [myid:3] - WARN > [RecvWorker:1:QuorumCnxManager$RecvWorker@1025] - Connection broken for id > 1, my id = 3, error = > java.io.EOFException > at java.io.DataInputStream.readInt(DataInputStream.java:392) > at > > org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:1010) > 2020-04-21 05:02:00,120 [myid:3] - WARN > [RecvWorker:1:QuorumCnxManager$RecvWorker@1028] - Interrupting SendWorker > 2020-04-21 05:02:00,143 [myid:3] - WARN > [SendWorker:1:QuorumCnxManager$SendWorker@941] - Interrupted while waiting > for message on queue > java.lang.InterruptedException > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014) > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2088) > at > java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:418) > at > > org.apache.zookeeper.server.quorum.QuorumCnxManager.pollSendQueue(QuorumCnxManager.java:1094) > at > > org.apache.zookeeper.server.quorum.QuorumCnxManager.access$700(QuorumCnxManager.java:74) > at > > org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:929) > > 2020-04-21 05:02:00,143 [myid:3] - WARN > [SendWorker:1:QuorumCnxManager$SendWorker@951] - Send worker leaving > thread > java.lang.InterruptedException > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014) > at > >
ZK connection broken to one server shutting down entire quorum
Hi team, During a recent patching for our ZK quorum, we experienced an unrecoverable outage. We have performed patches like this many times previously and is working fine in other environments of ours. The goal was to shut down each server one by one and provide patch updates then restart. However, this time, when zoo1 (follower) was shut down, the leader (zoo3) shutdown connection with remaining follower (zoo2) as well. What would cause the entire quorum to shutdown and not recover due to stopping only zoo1? Running zookeeper 3.4.14 in docker containers. zk_version 3.4.14-4c25d480e66aadd371de8bd2fd8da255ac140bcf, built on 03/06/2019 16:18 GMT *Config for master nodes:* tickTime=2000 maxClientCnxns=0 dataDir=/data dataLogDir=/datalog clientPort=2181 secureClientPort=2281 initLimit=10 syncLimit=5 autopurge.snapRetainCount=10 autopurge.purgeInterval=24 server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888 server.4=zoo4:2888:3888:observer server.5=zoo5:2888:3888:observer server.6=zoo6:2888:3888:observer *Config for observer nodes:* tickTime=2000 maxClientCnxns=0 dataDir=/data dataLogDir=/datalog clientPort=2181 secureClientPort=2281 initLimit=10 syncLimit=5 autopurge.snapRetainCount=10 autopurge.purgeInterval=24 peerType=observer server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888 server.4=zoo4:2888:3888:observer server.5=zoo5:2888:3888:observer server.6=zoo6:2888:3888:observer zoo1-zoo6 are the FQDNs of each server. Shutdown of zoo1 and quorum outage at 05:02 UTC *Logs on zoo3 (leader):* 2020-04-21 05:02:00,120 [myid:3] - WARN [RecvWorker:1:QuorumCnxManager$RecvWorker@1028] - Interrupting SendWorker java.io.EOFException at java.io.DataInputStream.readInt(DataInputStream.java:392) at org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:1010) 2020-04-21 05:02:00,096 [myid:3] - WARN [RecvWorker:1:QuorumCnxManager$RecvWorker@1025] - Connection broken for id 1, my id = 3, error = java.io.EOFException at java.io.DataInputStream.readInt(DataInputStream.java:392) at org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:1010) 2020-04-21 05:02:00,120 [myid:3] - WARN [RecvWorker:1:QuorumCnxManager$RecvWorker@1028] - Interrupting SendWorker 2020-04-21 05:02:00,143 [myid:3] - WARN [SendWorker:1:QuorumCnxManager$SendWorker@941] - Interrupted while waiting for message on queue java.lang.InterruptedException at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2088) at java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:418) at org.apache.zookeeper.server.quorum.QuorumCnxManager.pollSendQueue(QuorumCnxManager.java:1094) at org.apache.zookeeper.server.quorum.QuorumCnxManager.access$700(QuorumCnxManager.java:74) at org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:929) 2020-04-21 05:02:00,143 [myid:3] - WARN [SendWorker:1:QuorumCnxManager$SendWorker@951] - Send worker leaving thread java.lang.InterruptedException at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2088) at java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:418) at org.apache.zookeeper.server.quorum.QuorumCnxManager.pollSendQueue(QuorumCnxManager.java:1094) at org.apache.zookeeper.server.quorum.QuorumCnxManager.access$700(QuorumCnxManager.java:74) at org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:929) java.lang.InterruptedException at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2088) at java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:418) at org.apache.zookeeper.server.quorum.QuorumCnxManager.pollSendQueue(QuorumCnxManager.java:1094) at org.apache.zookeeper.server.quorum.QuorumCnxManager.access$700(QuorumCnxManager.java:74) at org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:929) 2020-04-21 05:02:00,143 [myid:3] - WARN [SendWorker:1:QuorumCnxManager$SendWorker@951] - Send worker leaving thread 2020-04-21 05:02:00,185 [myid:3] - ERROR [LearnerHandler-/:40504:LearnerHandler@648] - Unexpected exception causing shutdown while sock still open java.net.SocketException: Connection reset at java.net.SocketInputStream.read(SocketInputStream.java:210) at