You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@zookeeper.apache.org by Deepak Jagtap <de...@maxta.com> on 2014/02/12 03:26:58 UTC

zookeeper quorum falling apart with continuous leader election

Hi ,

I have 3 node zookeeper 3.5.0.1458648 quorum on my setup.
We came across a situation where one of the zk server in the cluster went
down
due to bad disk.
We observed that leader election keeps running in loop (starts, completes
and again starts). The loop repeats every couple of minutes.
Even restarting zookeeper server on both nodes doesn't help recovering from
this loop.
Network connection looks fine though, as I could telnet leader election
port and ssh from one node to other.
zookeeper client on each node is using "127.0.0.1:2181" as quorum string
for connecting to server, therefore if local zookeeper server is down
client app is dead.

I have uploaded zookeeper.log for both nodes at following link:
https://dl.dropboxusercontent.com/u/36429721/zkSupportLog.tar.gz

Any idea what might be wrong with the quorum? Please note that restarting
zookeeper server on both nodes doesn't help to recover from this situations.

Thanks & Regards,
Deepak

Re: zookeeper quorum falling apart with continuous leader election

Posted by Deepak Jagtap <de...@maxta.com>.
Thanks Flavio for quick repsonse!
Yes, logs for third node with bad disk are missing.

Hi Kishore, zk client completely lost the connection to zookeeper so most
probably there are no writes on the system. But on node 2 (Leader) I saw
some log message indicating successful session establishment from client,
hence possibility of writes on the server can not be ignored.

Thanks & Regards,
Deepak



On Wed, Feb 12, 2014 at 9:56 AM, kishore g <g....@gmail.com> wrote:

> Just for my understanding what do these messages indicate.  Also I see that
> n.zxid keeps incrementing, does it mean the system is accepting writes?
>
> node 2
> 2014-02-10 19:49:06,860 [myid:235] - INFO
> [WorkerReceiver[myid=235]:FastLeaderElection@594] - Notification: 234
> (n.leader), 0x4afe00000001 (n.zxid), 0x4b00 (n.round), LOOKING (n.state),
> 234 (n.sid), 0x4aff (n.peerEPoch), LOOKING (my state)1 (n.config version)
>
> node 1
> 2014-02-10 19:42:02,936 [myid:234] - INFO
> [WorkerReceiver[myid=234]:FastLeaderElection@594] - Notification: 234
> (n.leader), 0x4afa00000001 (n.zxid), 0x4afc (n.round), LOOKING (n.state),
> 234 (n.sid), 0x4afb (n.peerEPoch), LOOKING (my state)1 (n.config version)
>
>
>
>
> On Wed, Feb 12, 2014 at 6:47 AM, Flavio Junqueira <fpjunqueira@yahoo.com
> >wrote:
>
> > It sounds like LE is completing periodically, but the servers are not
> > being able to complete the synchronization step. We are also getting this
> > connection refused exception when the follower is trying to connect. This
> > is what I spotted for the follower:
> >
> > 2014-02-10 18:54:04,414 [myid:234] - INFO
> >  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Follower@65] - FOLLOWING -
> > LEADER ELECTION TOOK - 1
> > 2014-02-10 18:54:04,415 [myid:234] - WARN
> >  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Learner@239] - Unexpected
> > exception, tries=0, connecting to 10.0.57.235/10.0.57.235:2888
> > java.net.ConnectException: Connection refused
> >         at java.net.PlainSocketImpl.socketConnect(Native Method)
> >         at java.net.AbstractPlainSocketImpl.doConnect(Unknown Source)
> >         at java.net.AbstractPlainSocketImpl.connectToAddress(Unknown
> > Source)
> >         at java.net.AbstractPlainSocketImpl.connect(Unknown Source)
> >         at java.net.SocksSocketImpl.connect(Unknown Source)
> >         at java.net.Socket.connect(Unknown Source)
> >         at
> >
> org.apache.zookeeper.server.quorum.Learner.connectToLeader(Learner.java:231)
> >         at
> >
> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:73)
> >         at
> > org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:936)
> >
> > and this:
> >
> > 2014-02-10 18:55:05,508 [myid:234] - INFO
> >  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Learner@442] - Learner
> > received UPTODATE message
> > 2014-02-10 18:55:05,508 [myid:234] - WARN
> >  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Follower@92] - Exception
> when
> > following the leader
> > java.net.SocketException: Broken pipe
> >         at java.net.SocketOutputStream.socketWrite0(Native Method)
> >         at java.net.SocketOutputStream.socketWrite(Unknown Source)
> >         at java.net.SocketOutputStream.write(Unknown Source)
> >         at java.io.BufferedOutputStream.flushBuffer(Unknown Source)
> >         at java.io.BufferedOutputStream.flush(Unknown Source)
> >         at
> > org.apache.zookeeper.server.quorum.Learner.writePacket(Learner.java:145)
> >         at
> >
> org.apache.zookeeper.server.quorum.Learner.syncWithLeader(Learner.java:477)
> >         at
> >
> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:85)
> >         at
> > org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:936)
> >
> > On the leader side, we have this:
> >
> > 2014-02-10 19:48:03,705 [myid:235] - INFO
> >  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@328] - Synchronizing
> > with Follower sid: 234 maxCommittedLog=0x4afe00000001
> > minCommittedLog=0x4afe00000001 peerLastZxid=0x4afd00000001
> > 2014-02-10 19:48:03,705 [myid:235] - WARN
> >  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@389] - Unhandled
> > proposal scenario
> > 2014-02-10 19:48:03,705 [myid:235] - INFO
> >  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@404] - Sending SNAP
> > 2014-02-10 19:48:03,705 [myid:235] - INFO
> >  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@435] - Sending
> > snapshot last zxid of peer is 0x4afd00000001  zxid of leader is
> > 0x4aff00000000sent zxid of db as 0x4afe00000001
> > 2014-02-10 19:48:03,724 [myid:235] - WARN
> >  [LearnerHandler-/10.0.57.234:58829:Leader@698] - Commiting zxid
> > 0x4aff00000000 from /10.0.57.235:2888 not first!
> >
> > There are a couple of odd warnings there. Just to confirm, the node
> > missing in the logs is the one with the bad disk, right?
> >
> > -Flavio
> >
> > On 12 Feb 2014, at 02:26, Deepak Jagtap <de...@maxta.com> wrote:
> >
> > > Hi ,
> > >
> > > I have 3 node zookeeper 3.5.0.1458648 quorum on my setup.
> > > We came across a situation where one of the zk server in the cluster
> went
> > > down
> > > due to bad disk.
> > > We observed that leader election keeps running in loop (starts,
> completes
> > > and again starts). The loop repeats every couple of minutes.
> > > Even restarting zookeeper server on both nodes doesn't help recovering
> > from
> > > this loop.
> > > Network connection looks fine though, as I could telnet leader election
> > > port and ssh from one node to other.
> > > zookeeper client on each node is using "127.0.0.1:2181" as quorum
> string
> > > for connecting to server, therefore if local zookeeper server is down
> > > client app is dead.
> > >
> > > I have uploaded zookeeper.log for both nodes at following link:
> > > https://dl.dropboxusercontent.com/u/36429721/zkSupportLog.tar.gz
> > >
> > > Any idea what might be wrong with the quorum? Please note that
> restarting
> > > zookeeper server on both nodes doesn't help to recover from this
> > situations.
> > >
> > > Thanks & Regards,
> > > Deepak
> >
> >
>

Re: zookeeper quorum falling apart with continuous leader election

Posted by kishore g <g....@gmail.com>.
Just for my understanding what do these messages indicate.  Also I see that
n.zxid keeps incrementing, does it mean the system is accepting writes?

node 2
2014-02-10 19:49:06,860 [myid:235] - INFO
[WorkerReceiver[myid=235]:FastLeaderElection@594] - Notification: 234
(n.leader), 0x4afe00000001 (n.zxid), 0x4b00 (n.round), LOOKING (n.state),
234 (n.sid), 0x4aff (n.peerEPoch), LOOKING (my state)1 (n.config version)

node 1
2014-02-10 19:42:02,936 [myid:234] - INFO
[WorkerReceiver[myid=234]:FastLeaderElection@594] - Notification: 234
(n.leader), 0x4afa00000001 (n.zxid), 0x4afc (n.round), LOOKING (n.state),
234 (n.sid), 0x4afb (n.peerEPoch), LOOKING (my state)1 (n.config version)




On Wed, Feb 12, 2014 at 6:47 AM, Flavio Junqueira <fp...@yahoo.com>wrote:

> It sounds like LE is completing periodically, but the servers are not
> being able to complete the synchronization step. We are also getting this
> connection refused exception when the follower is trying to connect. This
> is what I spotted for the follower:
>
> 2014-02-10 18:54:04,414 [myid:234] - INFO
>  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Follower@65] - FOLLOWING -
> LEADER ELECTION TOOK - 1
> 2014-02-10 18:54:04,415 [myid:234] - WARN
>  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Learner@239] - Unexpected
> exception, tries=0, connecting to 10.0.57.235/10.0.57.235:2888
> java.net.ConnectException: Connection refused
>         at java.net.PlainSocketImpl.socketConnect(Native Method)
>         at java.net.AbstractPlainSocketImpl.doConnect(Unknown Source)
>         at java.net.AbstractPlainSocketImpl.connectToAddress(Unknown
> Source)
>         at java.net.AbstractPlainSocketImpl.connect(Unknown Source)
>         at java.net.SocksSocketImpl.connect(Unknown Source)
>         at java.net.Socket.connect(Unknown Source)
>         at
> org.apache.zookeeper.server.quorum.Learner.connectToLeader(Learner.java:231)
>         at
> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:73)
>         at
> org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:936)
>
> and this:
>
> 2014-02-10 18:55:05,508 [myid:234] - INFO
>  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Learner@442] - Learner
> received UPTODATE message
> 2014-02-10 18:55:05,508 [myid:234] - WARN
>  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Follower@92] - Exception when
> following the leader
> java.net.SocketException: Broken pipe
>         at java.net.SocketOutputStream.socketWrite0(Native Method)
>         at java.net.SocketOutputStream.socketWrite(Unknown Source)
>         at java.net.SocketOutputStream.write(Unknown Source)
>         at java.io.BufferedOutputStream.flushBuffer(Unknown Source)
>         at java.io.BufferedOutputStream.flush(Unknown Source)
>         at
> org.apache.zookeeper.server.quorum.Learner.writePacket(Learner.java:145)
>         at
> org.apache.zookeeper.server.quorum.Learner.syncWithLeader(Learner.java:477)
>         at
> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:85)
>         at
> org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:936)
>
> On the leader side, we have this:
>
> 2014-02-10 19:48:03,705 [myid:235] - INFO
>  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@328] - Synchronizing
> with Follower sid: 234 maxCommittedLog=0x4afe00000001
> minCommittedLog=0x4afe00000001 peerLastZxid=0x4afd00000001
> 2014-02-10 19:48:03,705 [myid:235] - WARN
>  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@389] - Unhandled
> proposal scenario
> 2014-02-10 19:48:03,705 [myid:235] - INFO
>  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@404] - Sending SNAP
> 2014-02-10 19:48:03,705 [myid:235] - INFO
>  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@435] - Sending
> snapshot last zxid of peer is 0x4afd00000001  zxid of leader is
> 0x4aff00000000sent zxid of db as 0x4afe00000001
> 2014-02-10 19:48:03,724 [myid:235] - WARN
>  [LearnerHandler-/10.0.57.234:58829:Leader@698] - Commiting zxid
> 0x4aff00000000 from /10.0.57.235:2888 not first!
>
> There are a couple of odd warnings there. Just to confirm, the node
> missing in the logs is the one with the bad disk, right?
>
> -Flavio
>
> On 12 Feb 2014, at 02:26, Deepak Jagtap <de...@maxta.com> wrote:
>
> > Hi ,
> >
> > I have 3 node zookeeper 3.5.0.1458648 quorum on my setup.
> > We came across a situation where one of the zk server in the cluster went
> > down
> > due to bad disk.
> > We observed that leader election keeps running in loop (starts, completes
> > and again starts). The loop repeats every couple of minutes.
> > Even restarting zookeeper server on both nodes doesn't help recovering
> from
> > this loop.
> > Network connection looks fine though, as I could telnet leader election
> > port and ssh from one node to other.
> > zookeeper client on each node is using "127.0.0.1:2181" as quorum string
> > for connecting to server, therefore if local zookeeper server is down
> > client app is dead.
> >
> > I have uploaded zookeeper.log for both nodes at following link:
> > https://dl.dropboxusercontent.com/u/36429721/zkSupportLog.tar.gz
> >
> > Any idea what might be wrong with the quorum? Please note that restarting
> > zookeeper server on both nodes doesn't help to recover from this
> situations.
> >
> > Thanks & Regards,
> > Deepak
>
>

Re: zookeeper quorum falling apart with continuous leader election

Posted by Flavio Junqueira <fp...@yahoo.com>.
It sounds like LE is completing periodically, but the servers are not being able to complete the synchronization step. We are also getting this connection refused exception when the follower is trying to connect. This is what I spotted for the follower:

2014-02-10 18:54:04,414 [myid:234] - INFO  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Follower@65] - FOLLOWING - LEADER ELECTION TOOK - 1
2014-02-10 18:54:04,415 [myid:234] - WARN  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Learner@239] - Unexpected exception, tries=0, connecting to 10.0.57.235/10.0.57.235:2888
java.net.ConnectException: Connection refused
	at java.net.PlainSocketImpl.socketConnect(Native Method)
	at java.net.AbstractPlainSocketImpl.doConnect(Unknown Source)
	at java.net.AbstractPlainSocketImpl.connectToAddress(Unknown Source)
	at java.net.AbstractPlainSocketImpl.connect(Unknown Source)
	at java.net.SocksSocketImpl.connect(Unknown Source)
	at java.net.Socket.connect(Unknown Source)
	at org.apache.zookeeper.server.quorum.Learner.connectToLeader(Learner.java:231)
	at org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:73)
	at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:936)

and this:

2014-02-10 18:55:05,508 [myid:234] - INFO  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Learner@442] - Learner received UPTODATE message
2014-02-10 18:55:05,508 [myid:234] - WARN  [QuorumPeer[myid=234]/0:0:0:0:0:0:0:0:2181:Follower@92] - Exception when following the leader
java.net.SocketException: Broken pipe
	at java.net.SocketOutputStream.socketWrite0(Native Method)
	at java.net.SocketOutputStream.socketWrite(Unknown Source)
	at java.net.SocketOutputStream.write(Unknown Source)
	at java.io.BufferedOutputStream.flushBuffer(Unknown Source)
	at java.io.BufferedOutputStream.flush(Unknown Source)
	at org.apache.zookeeper.server.quorum.Learner.writePacket(Learner.java:145)
	at org.apache.zookeeper.server.quorum.Learner.syncWithLeader(Learner.java:477)
	at org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:85)
	at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:936)

On the leader side, we have this:

2014-02-10 19:48:03,705 [myid:235] - INFO  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@328] - Synchronizing with Follower sid: 234 maxCommittedLog=0x4afe00000001 minCommittedLog=0x4afe00000001 peerLastZxid=0x4afd00000001
2014-02-10 19:48:03,705 [myid:235] - WARN  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@389] - Unhandled proposal scenario
2014-02-10 19:48:03,705 [myid:235] - INFO  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@404] - Sending SNAP
2014-02-10 19:48:03,705 [myid:235] - INFO  [LearnerHandler-/10.0.57.234:58829:LearnerHandler@435] - Sending snapshot last zxid of peer is 0x4afd00000001  zxid of leader is 0x4aff00000000sent zxid of db as 0x4afe00000001
2014-02-10 19:48:03,724 [myid:235] - WARN  [LearnerHandler-/10.0.57.234:58829:Leader@698] - Commiting zxid 0x4aff00000000 from /10.0.57.235:2888 not first!

There are a couple of odd warnings there. Just to confirm, the node missing in the logs is the one with the bad disk, right?

-Flavio

On 12 Feb 2014, at 02:26, Deepak Jagtap <de...@maxta.com> wrote:

> Hi ,
> 
> I have 3 node zookeeper 3.5.0.1458648 quorum on my setup.
> We came across a situation where one of the zk server in the cluster went
> down
> due to bad disk.
> We observed that leader election keeps running in loop (starts, completes
> and again starts). The loop repeats every couple of minutes.
> Even restarting zookeeper server on both nodes doesn't help recovering from
> this loop.
> Network connection looks fine though, as I could telnet leader election
> port and ssh from one node to other.
> zookeeper client on each node is using "127.0.0.1:2181" as quorum string
> for connecting to server, therefore if local zookeeper server is down
> client app is dead.
> 
> I have uploaded zookeeper.log for both nodes at following link:
> https://dl.dropboxusercontent.com/u/36429721/zkSupportLog.tar.gz
> 
> Any idea what might be wrong with the quorum? Please note that restarting
> zookeeper server on both nodes doesn't help to recover from this situations.
> 
> Thanks & Regards,
> Deepak