In my 3 node cluster if I stop one of the followers Zookeeper service
everything is fine, connections migrate and there's no interruption of
service.
If I stop the leader's Zookeeper service a lot of things *appear* fine, if
I run
echo "status" | nc localhost 2181
The output looks normal: a new node has been elected leader and
connections have moved. Logs aren't showing anything out of the ordinary.
As far as you can tell from looking at the Zookeeper cluster at this point
everything should be fine.
However on the client side there's a different story (client is an Apache
NiFi 1.9.2 cluster). The Curator client is in an infinite loop of:
o.a.c.f.state.ConnectionStateManager State change: RECONNECTED
o.a.c.f.state.ConnectionStateManager State change: SUSPENDED
o.a.c.f.imps.CuratorFrameworkImpl Background operation retry gave up
org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss
o.a.c.f.state.ConnectionStateManager State change: LOST
... series of attempts to reconnect which ultimately start the cycle over
again with ...
o.a.c.f.state.ConnectionStateManager State change: RECONNECTED
So while the Zookeeper cluster appears functional it is not actually
accepting client connections. I spent the lot of time looking at the
client code but what makes me confident it's was a Zookeeper issue is that
when I try to start the Zookeeper service on the former leader again and run
'echo "status" | nc localhost 2181'
It reports
This ZooKeeper instance is not currently serving requests
This is exactly the behavior described in the ZOOKEEPER-2164 comment trail
by Michael Durr.
The old leader now restarted has the following on loop in the logs:
2019-10-29 13:04:18,961 [myid:2] - INFO
[QuorumPeer[myid=2](plain=/0:0:0:0:0:0:0:0:2181)(secure=disabled):FastLeaderElection@919]
- Notification time out: 51200
2019-10-29 13:04:18,966 [myid:2] - INFO
[WorkerSender[myid=2]:QuorumCnxManager@430] - Have smaller server
identifier, so dropping the connection: (3, 2)
2019-10-29 13:04:18,966 [myid:2] - INFO
[WorkerReceiver[myid=2]:FastLeaderElection@679] - Notification: 2 (message
format version), 3 (n.leader), 0x26000000c0 (n.zxid), 0x2 (n.round),
LOOKING (n.state), 2 (n.sid), 0x27 (n.peerEPoch), LOOKING (my state)0
(n.config version)
2019-10-29 13:04:18,969 [myid:2] - INFO
[WorkerReceiver[myid=2]:FastLeaderElection@679] - Notification: 2 (message
format version), 3 (n.leader), 0x26000000c0 (n.zxid), 0x2 (n.round),
FOLLOWING (n.state), 1 (n.sid), 0x28 (n.peerEPoch), LOOKING (my state)0
(n.config version)
It never comes on as a follower and continues to report it is not serving
requests.
The new leader's logs have this series of events:
2019-10-29 13:09:10,221 [myid:3] - INFO [/0.0.0.0:3888
:QuorumCnxManager$Listener@888] - Received connection request /
10.251.0.7:43492
2019-10-29 13:09:10,221 [myid:3] - WARN
[RecvWorker:3:QuorumCnxManager$RecvWorker@1176] - Connection broken for id
3, my id = 3, error =
java.io.EOFException
at
java.base/java.io.DataInputStream.readInt(DataInputStream.java:397)
at
org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:1161)
2019-10-29 13:09:10,222 [myid:3] - WARN
[RecvWorker:3:QuorumCnxManager$RecvWorker@1179] - Interrupting SendWorker
2019-10-29 13:09:10,222 [myid:3] - WARN
[RecvWorker:2:QuorumCnxManager$RecvWorker@1176] - Connection broken for id
2, my id = 3, error =
java.net.SocketException: Socket closed
at java.base/java.net.SocketInputStream.socketRead0(Native Method)
at
java.base/java.net.SocketInputStream.socketRead(SocketInputStream.java:115)
at
java.base/java.net.SocketInputStream.read(SocketInputStream.java:168)
at
java.base/java.net.SocketInputStream.read(SocketInputStream.java:140)
at
java.base/java.io.BufferedInputStream.fill(BufferedInputStream.java:252)
at
java.base/java.io.BufferedInputStream.read(BufferedInputStream.java:271)
at
java.base/java.io.DataInputStream.readInt(DataInputStream.java:392)
at
org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:1161)
2019-10-29 13:09:10,223 [myid:3] - WARN
[RecvWorker:2:QuorumCnxManager$RecvWorker@1179] - Interrupting SendWorker
2019-10-29 13:09:10,221 [myid:3] - WARN
[SendWorker:2:QuorumCnxManager$SendWorker@1092] - Interrupted while
waiting for message on queue
java.lang.InterruptedException
at
java.base/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2056)
at
java.base/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2133)
at
java.base/java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:432)
at
org.apache.zookeeper.server.quorum.QuorumCnxManager.pollSendQueue(QuorumCnxManager.java:1243)
at
org.apache.zookeeper.server.quorum.QuorumCnxManager.access$700(QuorumCnxManager.java:78)
at
org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:1080)
2019-10-29 13:09:10,224 [myid:3] - WARN
[SendWorker:2:QuorumCnxManager$SendWorker@1102] - Send worker leaving
thread id 2 my id = 3
2019-10-29 13:09:10,223 [myid:3] - WARN
[SendWorker:3:QuorumCnxManager$SendWorker@1092] - Interrupted while
waiting for message on queue
java.lang.InterruptedException
at
java.base/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2056)
at
java.base/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2133)
at
java.base/java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:432)
at
org.apache.zookeeper.server.quorum.QuorumCnxManager.pollSendQueue(QuorumCnxManager.java:1243)
at
org.apache.zookeeper.server.quorum.QuorumCnxManager.access$700(QuorumCnxManager.java:78)
at
org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:1080)
2019-10-29 13:09:10,224 [myid:3] - WARN
[SendWorker:3:QuorumCnxManager$SendWorker@1102] - Send worker leaving
thread id 3 my id = 3
2019-10-29 13:09:10,226 [myid:3] - INFO [/0.0.0.0:3888
:QuorumCnxManager$Listener@888] - Received connection request /
10.251.0.6:57308
2019-10-29 13:09:10,231 [myid:3] - INFO
[WorkerReceiver[myid=3]:FastLeaderElection@679] - Notification: 2 (message
format version), 3 (n.leader), 0x26000000c0 (n.zxid), 0x2 (n.round),
LOOKING (n.state), 3 (n.sid), 0x27 (n.peerEPoch), LEADING (my state)0
(n.config version)
2019-10-29 13:09:10,231 [myid:3] - INFO
[WorkerReceiver[myid=3]:FastLeaderElection@679] - Notification: 2 (message
format version), 3 (n.leader), 0x26000000c0 (n.zxid), 0x2 (n.round),
LEADING (n.state), 3 (n.sid), 0x28 (n.peerEPoch), LEADING (my state)0
(n.config version)
The 3rd node that's always been a follower's logs just say on loop:
2019-10-29 13:15:10,286 [myid:1] - INFO
[WorkerReceiver[myid=1]:FastLeaderElection@679] - Notification: 2 (message
format version), 3 (n.leader), 0x26000000c0 (n.zxid), 0x2 (n.round),
LOOKING (n.state), 2 (n.sid), 0x27 (n.peerEPoch), FOLLOWING (my state)0
(n.config version)
I'm not sure what's happening here, but the cluster never recovers.
*TL;DR:* Sorry that's a lot of info but ultimately this was very easy to
reproduce: create a 3 node cluster, stop the leader, start the leader
again, and that server never successfully rejoins the cluster. This is at
version 3.5.5.
Thanks,
Matthew Knight
On Mon, Oct 28, 2019 at 5:57 PM Enrico Olivelli <[email protected]> wrote:
> Matthew,
> can you share some logs ?
> That issue is quite generic, and it describes a case that usually works as
> expected (= system works fine)
>
> It is very common to bounce a server in a cluster of three, so there must
> be a cause for your problem.
> In a 3 nodes clusters usually you can continue working until up to 2
> servers are up and running.
>
> Enrico
>
> Il giorno lun 28 ott 2019 alle ore 19:55 Matthew Knight <
> [email protected]> ha scritto:
>
> > Hi,
> >
> > I am setting up a 3-node Zookeeper cluster and have encountered the issue
> > described in https://issues.apache.org/jira/browse/ZOOKEEPER-2164
> > It seems like Zookeeper is unable to recover if the leader node shuts
> down
> > unexpectedly. Is this truly a general problem with Zookeeper? Is there
> a
> > way to configure/work around this issue besides restarting all of the
> > Zookeeper nodes in the cluster?
> >
> > Thanks,
> > Matthew Knight
> >
>