You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@zookeeper.apache.org by Austin Shoemaker <au...@cooliris.com> on 2008/09/02 18:56:57 UTC

Leader election stalled

Hi,

We have run into a situation where killing the leader results in followers
perpetually trying to reelect that leader.

We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients connecting
at random. We kill the leader and observe the impact, monitoring a script
that repeatedly prints the responses to "ruok" and "stat". All servers
except the killed leader respond with "imok" and "ZooKeeperServer not
running", respectively.

About half of the time, each remaining server gets into a loop of failing to
connect to the killed leader and then reelecting the killed leader.

Here is an example log, which is representative of similar logs on the other
servers. We additionally logged connectivity during leader election. If
anyone would like complete logs, let me know.

Thanks,

Austin Shoemaker

WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
*WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
ERROR - [QuorumPeer:Follower@137] - FIXMSG
java.net.ConnectException: Connection refused
*
.... cont'd ....*

ERROR - [QuorumPeer:Follower@364] - FIXMSG
java.lang.Exception: shutdown Follower
        at
com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java:364)
        at
com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.22:2888
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.22:2888
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.21:2888
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.21:2888
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.12:2888
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.12:2888
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.11:2888
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.11:2888
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.12:2890
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.12:2890
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.11:2890
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.11:2890
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.22:2889
*WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred when
sending / receiving packet to / from /10.50.65.22:2889
java.net.SocketTimeoutException: Receive timed out
*WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to
/10.50.65.21:2890
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.21:2890
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.21:2889
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.21:2889
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.12:2889
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.12:2889
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
10.50.65.11:2889
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.11:2889
WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
result.winner = 7
*WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete, address
= /10.50.65.22:2889
WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
ERROR - [QuorumPeer:Follower@137] - FIXMSG
java.net.ConnectException: Connection refused
*        at java.net.PlainSocketImpl.socketConnect(Native Method)
        at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
        at
java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
        at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
        at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
        at java.net.Socket.connect(Socket.java:519)
        at
com.yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java:133)
        at
com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)

Re: Leader election stalled

Posted by Austin Shoemaker <au...@cooliris.com>.
Got it, thanks! I believe the problem still exists- please see my  
comment.

Best,
Austin

On Sep 16, 2008, at 3:26 AM, Flavio Junqueira wrote:

> Austin, Please check:
>
> https://issues.apache.org/jira/browse/ZOOKEEPER-140
>
> Thanks,
> -Flavio
>
>> -----Original Message-----
>> From: Austin Shoemaker [mailto:austin@cooliris.com]
>> Sent: Tuesday, September 16, 2008 12:22 PM
>> To: zookeeper-user@hadoop.apache.org
>> Subject: Re: Leader election stalled
>>
>> Ben,
>>
>> Here is a proposed fix for the deadlock issue in QuorumCnxManager.
>>
>> The protocol starts by an initiator invoking
>> handleConnection(socket_out) where socket is a connection to a remote
>> peer,
>> or if an incoming connection first triggers
>> handleConnection(socket_in) before we initiate a connection to the
>> peer. In the
>> event that we and the peer both initiate connections, the above calls
>> to handleConnection will proceed on different threads
>> in the same peer.
>>
>> Per-peer instance variables
>> myVersion = 0
>> myChallenge = genChallenge()
>>
>> "socket" is the connection to the peer.
>>
>> boolean handleConnection(socket) throws Exception {
>>     done = false
>>     wins = false
>>
>>     while (!done) {
>>         // Send the current version and challenge to the peer, then
>> wait for it to send its current version and challenge.
>>         // The read is blocking though we expect the peer to write
>> since reads and writes are matched.
>>         synchronized (challengeLock) {
>>     		socket.write(myVersion, myChallenge)
>>     	}
>>         peerVersion, peerChallenge = socket.read()
>>
>>         synchronized (challengeLock) {
>>             // If peer is obsolete, bring it up to date.
>>             if (peerVersion < myVersion) {
>>                 continue;
>>             }
>>
>>             // If we are obsolete, wait to be brought up to date.
>>             if (peerVersion > myVersion) {
>>                 myVersion = peerVersion
>>             	myChallenge = genChallenge()
>>             	continue
>>             }
>>
>>     	    assert(myVersion == peerVersion)
>>
>>             // Challenges are compared, resulting in win, lose, or
>> retry.
>>             if (myChallenge > peerChallenge) {
>>                 wins = true
>>                 done = true
>>             } else if (myChallenge < peerChallenge) {
>>                 done = true
>>             } else {
>>                 ++myVersion
>>                 myChallenge = genChallenge()
>>             }
>>         }
>>     }
>>
>>     // We return true if we won, otherwise we return false. Either we
>> or the peer will win, not both. If a connection error occurs,
>>     // this method will throw an exception.
>>     return wins
>> }
>>
>> Do you think it's correct? I wonder if there is a way to simplify  
>> this
>> protocol.
>>
>> Austin
>>
>> On Sep 12, 2008, at 4:51 PM, Austin Shoemaker wrote:
>>
>>> Ben,
>>>
>>> I am able to run algorithm 3 successfully sometimes, though
>>> frequently the servers deadlock in
>>> QuorumCnxManager:initiateConnection on s.read(msgBuffer) when
>>> reading the challenge from the peer.
>>>
>>> Calls to initiateConnection and receiveConnection are synchronized,
>>> so only one or the other can be executing at a time. This prevents
>>> two connections from opening between the same pair of servers.
>>>
>>> However, it seems that this leads to deadlock, as in this scenario:
>>>
>>> A (initiate --> B)
>>> B (initiate --> C)
>>> C (initiate --> A)
>>>
>>> initiateConnection can only complete when receiveConnection runs on
>>> the remote peer and answers the challenge. If all servers are
>>> blocked in initiateConnection, receiveConnection never runs and
>>> leader election halts.
>>>
>>> Looking forward to your thoughts.
>>>
>>> Thanks,
>>>
>>> Austin
>>>
>>> On Sep 2, 2008, at 10:14 AM, Benjamin Reed wrote:
>>>
>>>> Austin,
>>>>
>>>> Could you try using the new leader election algorithm? You need to
>>>> set
>>>> the algorithm type to 3 and you also need to set the election port
>>>> (TCP)
>>>> to be used.
>>>>
>>>> See http://zookeeper.wiki.sourceforge.net/ZooKeeperConfiguration  
>>>> for
>>>> more details.
>>>>
>>>> ben
>>>>
>>>> -----Original Message-----
>>>> From: Austin Shoemaker [mailto:austin@cooliris.com]
>>>> Sent: Tuesday, September 02, 2008 9:57 AM
>>>> To: zookeeper-user@hadoop.apache.org
>>>> Subject: Leader election stalled
>>>>
>>>> Hi,
>>>>
>>>> We have run into a situation where killing the leader results in
>>>> followers
>>>> perpetually trying to reelect that leader.
>>>>
>>>> We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients
>>>> connecting
>>>> at random. We kill the leader and observe the impact, monitoring a
>>>> script
>>>> that repeatedly prints the responses to "ruok" and "stat". All
>>>> servers
>>>> except the killed leader respond with "imok" and "ZooKeeperServer  
>>>> not
>>>> running", respectively.
>>>>
>>>> About half of the time, each remaining server gets into a loop of
>>>> failing to
>>>> connect to the killed leader and then reelecting the killed leader.
>>>>
>>>> Here is an example log, which is representative of similar logs on
>>>> the
>>>> other
>>>> servers. We additionally logged connectivity during leader
>>>> election. If
>>>> anyone would like complete logs, let me know.
>>>>
>>>> Thanks,
>>>>
>>>> Austin Shoemaker
>>>>
>>>> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
>>>> *WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
>>>> ERROR - [QuorumPeer:Follower@137] - FIXMSG
>>>> java.net.ConnectException: Connection refused
>>>> *
>>>> .... cont'd ....*
>>>>
>>>> ERROR - [QuorumPeer:Follower@364] - FIXMSG
>>>> java.lang.Exception: shutdown Follower
>>>>      at
>>>> com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java:
>>>> 364)
>>>>      at
>>>> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java: 
>>>> 403)
>>>> WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.22:2888
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.22:2888
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.21:2888
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.21:2888
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.12:2888
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.12:2888
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.11:2888
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.11:2888
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.12:2890
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.12:2890
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.11:2890
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.11:2890
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.22:2889
>>>> *WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred
>>>> when
>>>> sending / receiving packet to / from /10.50.65.22:2889
>>>> java.net.SocketTimeoutException: Receive timed out
>>>> *WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to
>>>> /10.50.65.21:2890
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.21:2890
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.21:2889
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.21:2889
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.12:2889
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.12:2889
>>>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
>>>> packet
>>>> to /
>>>> 10.50.65.11:2889
>>>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
>>>> from /
>>>> 10.50.65.11:2889
>>>> WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
>>>> WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
>>>> WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
>>>> WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
>>>> WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
>>>> result.winner = 7
>>>> *WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete,
>>>> address
>>>> = /10.50.65.22:2889
>>>> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
>>>> WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
>>>> ERROR - [QuorumPeer:Follower@137] - FIXMSG
>>>> java.net.ConnectException: Connection refused
>>>> *        at java.net.PlainSocketImpl.socketConnect(Native Method)
>>>>      at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java: 
>>>> 333)
>>>>      at
>>>> java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>>>>      at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>>>>      at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>>>>      at java.net.Socket.connect(Socket.java:519)
>>>>      at
>>>> com
>>>> .yahoo 
>>>> .zookeeper.server.quorum.Follower.followLeader(Follower.java:13
>>>> 3)
>>>>      at
>>>> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java: 
>>>> 399)
>>>
>
>


RE: Leader election stalled

Posted by Flavio Junqueira <fp...@yahoo-inc.com>.
Austin, Please check:

https://issues.apache.org/jira/browse/ZOOKEEPER-140

Thanks,
-Flavio

> -----Original Message-----
> From: Austin Shoemaker [mailto:austin@cooliris.com]
> Sent: Tuesday, September 16, 2008 12:22 PM
> To: zookeeper-user@hadoop.apache.org
> Subject: Re: Leader election stalled
> 
> Ben,
> 
> Here is a proposed fix for the deadlock issue in QuorumCnxManager.
> 
> The protocol starts by an initiator invoking
> handleConnection(socket_out) where socket is a connection to a remote
> peer,
> or if an incoming connection first triggers
> handleConnection(socket_in) before we initiate a connection to the
> peer. In the
> event that we and the peer both initiate connections, the above calls
> to handleConnection will proceed on different threads
> in the same peer.
> 
> Per-peer instance variables
> myVersion = 0
> myChallenge = genChallenge()
> 
> "socket" is the connection to the peer.
> 
> boolean handleConnection(socket) throws Exception {
>      done = false
>      wins = false
> 
>      while (!done) {
>          // Send the current version and challenge to the peer, then
> wait for it to send its current version and challenge.
>          // The read is blocking though we expect the peer to write
> since reads and writes are matched.
>          synchronized (challengeLock) {
>      		socket.write(myVersion, myChallenge)
>      	}
>          peerVersion, peerChallenge = socket.read()
> 
>          synchronized (challengeLock) {
>              // If peer is obsolete, bring it up to date.
>              if (peerVersion < myVersion) {
>                  continue;
>              }
> 
>              // If we are obsolete, wait to be brought up to date.
>              if (peerVersion > myVersion) {
>                  myVersion = peerVersion
>              	myChallenge = genChallenge()
>              	continue
>              }
> 
>      	    assert(myVersion == peerVersion)
> 
>              // Challenges are compared, resulting in win, lose, or
> retry.
>              if (myChallenge > peerChallenge) {
>                  wins = true
>                  done = true
>              } else if (myChallenge < peerChallenge) {
>                  done = true
>              } else {
>                  ++myVersion
>                  myChallenge = genChallenge()
>              }
>          }
>      }
> 
>      // We return true if we won, otherwise we return false. Either we
> or the peer will win, not both. If a connection error occurs,
>      // this method will throw an exception.
>      return wins
> }
> 
> Do you think it's correct? I wonder if there is a way to simplify this
> protocol.
> 
> Austin
> 
> On Sep 12, 2008, at 4:51 PM, Austin Shoemaker wrote:
> 
> > Ben,
> >
> > I am able to run algorithm 3 successfully sometimes, though
> > frequently the servers deadlock in
> > QuorumCnxManager:initiateConnection on s.read(msgBuffer) when
> > reading the challenge from the peer.
> >
> > Calls to initiateConnection and receiveConnection are synchronized,
> > so only one or the other can be executing at a time. This prevents
> > two connections from opening between the same pair of servers.
> >
> > However, it seems that this leads to deadlock, as in this scenario:
> >
> > A (initiate --> B)
> > B (initiate --> C)
> > C (initiate --> A)
> >
> > initiateConnection can only complete when receiveConnection runs on
> > the remote peer and answers the challenge. If all servers are
> > blocked in initiateConnection, receiveConnection never runs and
> > leader election halts.
> >
> > Looking forward to your thoughts.
> >
> > Thanks,
> >
> > Austin
> >
> > On Sep 2, 2008, at 10:14 AM, Benjamin Reed wrote:
> >
> >> Austin,
> >>
> >> Could you try using the new leader election algorithm? You need to
> >> set
> >> the algorithm type to 3 and you also need to set the election port
> >> (TCP)
> >> to be used.
> >>
> >> See http://zookeeper.wiki.sourceforge.net/ZooKeeperConfiguration for
> >> more details.
> >>
> >> ben
> >>
> >> -----Original Message-----
> >> From: Austin Shoemaker [mailto:austin@cooliris.com]
> >> Sent: Tuesday, September 02, 2008 9:57 AM
> >> To: zookeeper-user@hadoop.apache.org
> >> Subject: Leader election stalled
> >>
> >> Hi,
> >>
> >> We have run into a situation where killing the leader results in
> >> followers
> >> perpetually trying to reelect that leader.
> >>
> >> We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients
> >> connecting
> >> at random. We kill the leader and observe the impact, monitoring a
> >> script
> >> that repeatedly prints the responses to "ruok" and "stat". All
> >> servers
> >> except the killed leader respond with "imok" and "ZooKeeperServer not
> >> running", respectively.
> >>
> >> About half of the time, each remaining server gets into a loop of
> >> failing to
> >> connect to the killed leader and then reelecting the killed leader.
> >>
> >> Here is an example log, which is representative of similar logs on
> >> the
> >> other
> >> servers. We additionally logged connectivity during leader
> >> election. If
> >> anyone would like complete logs, let me know.
> >>
> >> Thanks,
> >>
> >> Austin Shoemaker
> >>
> >> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
> >> *WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
> >> ERROR - [QuorumPeer:Follower@137] - FIXMSG
> >> java.net.ConnectException: Connection refused
> >> *
> >> .... cont'd ....*
> >>
> >> ERROR - [QuorumPeer:Follower@364] - FIXMSG
> >> java.lang.Exception: shutdown Follower
> >>       at
> >> com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java:
> >> 364)
> >>       at
> >> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
> >> WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.22:2888
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.22:2888
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.21:2888
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.21:2888
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.12:2888
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.12:2888
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.11:2888
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.11:2888
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.12:2890
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.12:2890
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.11:2890
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.11:2890
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.22:2889
> >> *WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred
> >> when
> >> sending / receiving packet to / from /10.50.65.22:2889
> >> java.net.SocketTimeoutException: Receive timed out
> >> *WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to
> >> /10.50.65.21:2890
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.21:2890
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.21:2889
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.21:2889
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.12:2889
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.12:2889
> >> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> >> packet
> >> to /
> >> 10.50.65.11:2889
> >> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response
> >> from /
> >> 10.50.65.11:2889
> >> WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
> >> WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
> >> WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
> >> WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
> >> WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
> >> result.winner = 7
> >> *WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete,
> >> address
> >> = /10.50.65.22:2889
> >> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
> >> WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
> >> ERROR - [QuorumPeer:Follower@137] - FIXMSG
> >> java.net.ConnectException: Connection refused
> >> *        at java.net.PlainSocketImpl.socketConnect(Native Method)
> >>       at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
> >>       at
> >> java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
> >>       at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
> >>       at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
> >>       at java.net.Socket.connect(Socket.java:519)
> >>       at
> >> com
> >> .yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java:13
> >> 3)
> >>       at
> >> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)
> >



Re: Leader election stalled

Posted by Austin Shoemaker <au...@cooliris.com>.
Ben,

Here is a proposed fix for the deadlock issue in QuorumCnxManager.

The protocol starts by an initiator invoking  
handleConnection(socket_out) where socket is a connection to a remote  
peer,
or if an incoming connection first triggers  
handleConnection(socket_in) before we initiate a connection to the  
peer. In the
event that we and the peer both initiate connections, the above calls  
to handleConnection will proceed on different threads
in the same peer.

Per-peer instance variables
myVersion = 0
myChallenge = genChallenge()

"socket" is the connection to the peer.

boolean handleConnection(socket) throws Exception {
     done = false
     wins = false

     while (!done) {
         // Send the current version and challenge to the peer, then  
wait for it to send its current version and challenge.
         // The read is blocking though we expect the peer to write  
since reads and writes are matched.
         synchronized (challengeLock) {
     		socket.write(myVersion, myChallenge)
     	}
         peerVersion, peerChallenge = socket.read()

         synchronized (challengeLock) {
             // If peer is obsolete, bring it up to date.
             if (peerVersion < myVersion) {
                 continue;
             }

             // If we are obsolete, wait to be brought up to date.
             if (peerVersion > myVersion) {
                 myVersion = peerVersion
             	myChallenge = genChallenge()
             	continue
             }
     	
     	    assert(myVersion == peerVersion)

             // Challenges are compared, resulting in win, lose, or  
retry.
             if (myChallenge > peerChallenge) {
                 wins = true
                 done = true
             } else if (myChallenge < peerChallenge) {
                 done = true
             } else {
                 ++myVersion
                 myChallenge = genChallenge()
             }
         }
     }

     // We return true if we won, otherwise we return false. Either we  
or the peer will win, not both. If a connection error occurs,
     // this method will throw an exception.
     return wins
}

Do you think it's correct? I wonder if there is a way to simplify this  
protocol.

Austin

On Sep 12, 2008, at 4:51 PM, Austin Shoemaker wrote:

> Ben,
>
> I am able to run algorithm 3 successfully sometimes, though  
> frequently the servers deadlock in  
> QuorumCnxManager:initiateConnection on s.read(msgBuffer) when  
> reading the challenge from the peer.
>
> Calls to initiateConnection and receiveConnection are synchronized,  
> so only one or the other can be executing at a time. This prevents  
> two connections from opening between the same pair of servers.
>
> However, it seems that this leads to deadlock, as in this scenario:
>
> A (initiate --> B)
> B (initiate --> C)
> C (initiate --> A)
>
> initiateConnection can only complete when receiveConnection runs on  
> the remote peer and answers the challenge. If all servers are  
> blocked in initiateConnection, receiveConnection never runs and  
> leader election halts.
>
> Looking forward to your thoughts.
>
> Thanks,
>
> Austin
>
> On Sep 2, 2008, at 10:14 AM, Benjamin Reed wrote:
>
>> Austin,
>>
>> Could you try using the new leader election algorithm? You need to  
>> set
>> the algorithm type to 3 and you also need to set the election port  
>> (TCP)
>> to be used.
>>
>> See http://zookeeper.wiki.sourceforge.net/ZooKeeperConfiguration for
>> more details.
>>
>> ben
>>
>> -----Original Message-----
>> From: Austin Shoemaker [mailto:austin@cooliris.com]
>> Sent: Tuesday, September 02, 2008 9:57 AM
>> To: zookeeper-user@hadoop.apache.org
>> Subject: Leader election stalled
>>
>> Hi,
>>
>> We have run into a situation where killing the leader results in
>> followers
>> perpetually trying to reelect that leader.
>>
>> We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients
>> connecting
>> at random. We kill the leader and observe the impact, monitoring a
>> script
>> that repeatedly prints the responses to "ruok" and "stat". All  
>> servers
>> except the killed leader respond with "imok" and "ZooKeeperServer not
>> running", respectively.
>>
>> About half of the time, each remaining server gets into a loop of
>> failing to
>> connect to the killed leader and then reelecting the killed leader.
>>
>> Here is an example log, which is representative of similar logs on  
>> the
>> other
>> servers. We additionally logged connectivity during leader  
>> election. If
>> anyone would like complete logs, let me know.
>>
>> Thanks,
>>
>> Austin Shoemaker
>>
>> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
>> *WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
>> ERROR - [QuorumPeer:Follower@137] - FIXMSG
>> java.net.ConnectException: Connection refused
>> *
>> .... cont'd ....*
>>
>> ERROR - [QuorumPeer:Follower@364] - FIXMSG
>> java.lang.Exception: shutdown Follower
>>       at
>> com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java: 
>> 364)
>>       at
>> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
>> WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.22:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.22:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.21:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.21:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.12:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.12:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.11:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.11:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.12:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.12:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.11:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.11:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.22:2889
>> *WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred  
>> when
>> sending / receiving packet to / from /10.50.65.22:2889
>> java.net.SocketTimeoutException: Receive timed out
>> *WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to
>> /10.50.65.21:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.21:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.21:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.21:2889
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.12:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.12:2889
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet
>> to /
>> 10.50.65.11:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.11:2889
>> WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
>> WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
>> WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
>> WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
>> WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
>> result.winner = 7
>> *WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete,
>> address
>> = /10.50.65.22:2889
>> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
>> WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
>> ERROR - [QuorumPeer:Follower@137] - FIXMSG
>> java.net.ConnectException: Connection refused
>> *        at java.net.PlainSocketImpl.socketConnect(Native Method)
>>       at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
>>       at
>> java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>>       at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>>       at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>>       at java.net.Socket.connect(Socket.java:519)
>>       at
>> com 
>> .yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java:13
>> 3)
>>       at
>> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)
>


Re: Leader election stalled

Posted by Austin Shoemaker <au...@cooliris.com>.
Ben,

I am able to run algorithm 3 successfully sometimes, though frequently  
the servers deadlock in QuorumCnxManager:initiateConnection on  
s.read(msgBuffer) when reading the challenge from the peer.

Calls to initiateConnection and receiveConnection are synchronized, so  
only one or the other can be executing at a time. This prevents two  
connections from opening between the same pair of servers.

However, it seems that this leads to deadlock, as in this scenario:

A (initiate --> B)
B (initiate --> C)
C (initiate --> A)

initiateConnection can only complete when receiveConnection runs on  
the remote peer and answers the challenge. If all servers are blocked  
in initiateConnection, receiveConnection never runs and leader  
election halts.

Looking forward to your thoughts.

Thanks,

Austin

On Sep 2, 2008, at 10:14 AM, Benjamin Reed wrote:

> Austin,
>
> Could you try using the new leader election algorithm? You need to set
> the algorithm type to 3 and you also need to set the election port  
> (TCP)
> to be used.
>
> See http://zookeeper.wiki.sourceforge.net/ZooKeeperConfiguration for
> more details.
>
> ben
>
> -----Original Message-----
> From: Austin Shoemaker [mailto:austin@cooliris.com]
> Sent: Tuesday, September 02, 2008 9:57 AM
> To: zookeeper-user@hadoop.apache.org
> Subject: Leader election stalled
>
> Hi,
>
> We have run into a situation where killing the leader results in
> followers
> perpetually trying to reelect that leader.
>
> We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients
> connecting
> at random. We kill the leader and observe the impact, monitoring a
> script
> that repeatedly prints the responses to "ruok" and "stat". All servers
> except the killed leader respond with "imok" and "ZooKeeperServer not
> running", respectively.
>
> About half of the time, each remaining server gets into a loop of
> failing to
> connect to the killed leader and then reelecting the killed leader.
>
> Here is an example log, which is representative of similar logs on the
> other
> servers. We additionally logged connectivity during leader election.  
> If
> anyone would like complete logs, let me know.
>
> Thanks,
>
> Austin Shoemaker
>
> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
> *WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
> ERROR - [QuorumPeer:Follower@137] - FIXMSG
> java.net.ConnectException: Connection refused
> *
> .... cont'd ....*
>
> ERROR - [QuorumPeer:Follower@364] - FIXMSG
> java.lang.Exception: shutdown Follower
>        at
> com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java:364)
>        at
> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
> WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.22:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.22:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.21:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.21:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.12:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.12:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.11:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.11:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.12:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.12:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.11:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.11:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.22:2889
> *WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred  
> when
> sending / receiving packet to / from /10.50.65.22:2889
> java.net.SocketTimeoutException: Receive timed out
> *WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to
> /10.50.65.21:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.21:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.21:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.21:2889
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.12:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.12:2889
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
> packet
> to /
> 10.50.65.11:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
> from /
> 10.50.65.11:2889
> WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
> WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
> WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
> WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
> WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
> result.winner = 7
> *WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete,
> address
> = /10.50.65.22:2889
> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
> WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
> ERROR - [QuorumPeer:Follower@137] - FIXMSG
> java.net.ConnectException: Connection refused
> *        at java.net.PlainSocketImpl.socketConnect(Native Method)
>        at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
>        at
> java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>        at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>        at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>        at java.net.Socket.connect(Socket.java:519)
>        at
> com 
> .yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java:13
> 3)
>        at
> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)


RE: Leader election stalled

Posted by Benjamin Reed <br...@yahoo-inc.com>.
Austin,

Could you try using the new leader election algorithm? You need to set
the algorithm type to 3 and you also need to set the election port (TCP)
to be used.

See http://zookeeper.wiki.sourceforge.net/ZooKeeperConfiguration for
more details.

ben

-----Original Message-----
From: Austin Shoemaker [mailto:austin@cooliris.com] 
Sent: Tuesday, September 02, 2008 9:57 AM
To: zookeeper-user@hadoop.apache.org
Subject: Leader election stalled

Hi,

We have run into a situation where killing the leader results in
followers
perpetually trying to reelect that leader.

We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients
connecting
at random. We kill the leader and observe the impact, monitoring a
script
that repeatedly prints the responses to "ruok" and "stat". All servers
except the killed leader respond with "imok" and "ZooKeeperServer not
running", respectively.

About half of the time, each remaining server gets into a loop of
failing to
connect to the killed leader and then reelecting the killed leader.

Here is an example log, which is representative of similar logs on the
other
servers. We additionally logged connectivity during leader election. If
anyone would like complete logs, let me know.

Thanks,

Austin Shoemaker

WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
*WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
ERROR - [QuorumPeer:Follower@137] - FIXMSG
java.net.ConnectException: Connection refused
*
.... cont'd ....*

ERROR - [QuorumPeer:Follower@364] - FIXMSG
java.lang.Exception: shutdown Follower
        at
com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java:364)
        at
com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.22:2888
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.22:2888
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.21:2888
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.21:2888
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.12:2888
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.12:2888
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.11:2888
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.11:2888
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.12:2890
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.12:2890
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.11:2890
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.11:2890
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.22:2889
*WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred when
sending / receiving packet to / from /10.50.65.22:2889
java.net.SocketTimeoutException: Receive timed out
*WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to
/10.50.65.21:2890
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.21:2890
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.21:2889
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.21:2889
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.12:2889
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.12:2889
WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet
to /
10.50.65.11:2889
WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
10.50.65.11:2889
WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
result.winner = 7
*WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete,
address
= /10.50.65.22:2889
WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
ERROR - [QuorumPeer:Follower@137] - FIXMSG
java.net.ConnectException: Connection refused
*        at java.net.PlainSocketImpl.socketConnect(Native Method)
        at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
        at
java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
        at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
        at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
        at java.net.Socket.connect(Socket.java:519)
        at
com.yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java:13
3)
        at
com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)

Re: Leader election stalled

Posted by Austin Shoemaker <au...@cooliris.com>.
We will retry with the new election algorithm and let you know the  
results.

Thanks for getting back so quickly.

Austin

On Sep 2, 2008, at 10:22 AM, Benjamin Reed wrote:

> I think there is a race condition that is probably easy to get into  
> with
> the old leader election and a large number of servers:
>
> 1) Leader dies
> 2) Followers start looking for a new leader before all Followers have
> abandoned the Leader
> 3) The Followers looking for a new leader see votes of Followers still
> following the (now dead) Leader and start voting for the dead Leader
> 4) The dead Leader gets reelected.
>
> For the old leader election a server should not vote for another  
> server
> that is not nominating himself.
>
> I'll open a Jira.
>
> ben
>
> -----Original Message-----
> From: Mahadev Konar [mailto:mahadev@yahoo-inc.com]
> Sent: Tuesday, September 02, 2008 10:06 AM
> To: zookeeper-user@hadoop.apache.org
> Subject: Re: Leader election stalled
>
> Hi Austin,
> Did you kill the leader process? It looks like that you didn't kill  
> the
> server since its responding to ruok. Is that true?
>
> mahadev
>
>
> On 9/2/08 9:56 AM, "Austin Shoemaker" <au...@cooliris.com> wrote:
>
>> Hi,
>>
>> We have run into a situation where killing the leader results in
> followers
>> perpetually trying to reelect that leader.
>>
>> We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients
> connecting
>> at random. We kill the leader and observe the impact, monitoring a
> script
>> that repeatedly prints the responses to "ruok" and "stat". All  
>> servers
>> except the killed leader respond with "imok" and "ZooKeeperServer not
>> running", respectively.
>>
>> About half of the time, each remaining server gets into a loop of
> failing to
>> connect to the killed leader and then reelecting the killed leader.
>>
>> Here is an example log, which is representative of similar logs on  
>> the
> other
>> servers. We additionally logged connectivity during leader election.
> If
>> anyone would like complete logs, let me know.
>>
>> Thanks,
>>
>> Austin Shoemaker
>>
>> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
>> *WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
>> ERROR - [QuorumPeer:Follower@137] - FIXMSG
>> java.net.ConnectException: Connection refused
>> *
>> .... cont'd ....*
>>
>> ERROR - [QuorumPeer:Follower@364] - FIXMSG
>> java.lang.Exception: shutdown Follower
>>        at
>> com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java: 
>> 364)
>>        at
>> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
>> WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.22:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.22:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.21:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.21:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.12:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.12:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.11:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.11:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.12:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.12:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.11:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.11:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.22:2889
>> *WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred
> when
>> sending / receiving packet to / from /10.50.65.22:2889
>> java.net.SocketTimeoutException: Receive timed out
>> *WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to
>> /10.50.65.21:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.21:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.21:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.21:2889
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.12:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.12:2889
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
> packet to /
>> 10.50.65.11:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from
> /
>> 10.50.65.11:2889
>> WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
>> WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
>> WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
>> WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
>> WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
>> result.winner = 7
>> *WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete,
> address
>> = /10.50.65.22:2889
>> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
>> WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
>> ERROR - [QuorumPeer:Follower@137] - FIXMSG
>> java.net.ConnectException: Connection refused
>> *        at java.net.PlainSocketImpl.socketConnect(Native Method)
>>        at
> java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
>>        at
>> java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>>        at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>>        at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>>        at java.net.Socket.connect(Socket.java:519)
>>        at
>>
> com 
> .yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java:13
> 3)
>>        at
>> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)
>


RE: Leader election stalled

Posted by Benjamin Reed <br...@yahoo-inc.com>.
I think there is a race condition that is probably easy to get into with
the old leader election and a large number of servers:

1) Leader dies
2) Followers start looking for a new leader before all Followers have
abandoned the Leader
3) The Followers looking for a new leader see votes of Followers still
following the (now dead) Leader and start voting for the dead Leader
4) The dead Leader gets reelected.

For the old leader election a server should not vote for another server
that is not nominating himself.

I'll open a Jira.

ben

-----Original Message-----
From: Mahadev Konar [mailto:mahadev@yahoo-inc.com] 
Sent: Tuesday, September 02, 2008 10:06 AM
To: zookeeper-user@hadoop.apache.org
Subject: Re: Leader election stalled

Hi Austin,
 Did you kill the leader process? It looks like that you didn't kill the
server since its responding to ruok. Is that true?

mahadev


On 9/2/08 9:56 AM, "Austin Shoemaker" <au...@cooliris.com> wrote:

> Hi,
> 
> We have run into a situation where killing the leader results in
followers
> perpetually trying to reelect that leader.
> 
> We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients
connecting
> at random. We kill the leader and observe the impact, monitoring a
script
> that repeatedly prints the responses to "ruok" and "stat". All servers
> except the killed leader respond with "imok" and "ZooKeeperServer not
> running", respectively.
> 
> About half of the time, each remaining server gets into a loop of
failing to
> connect to the killed leader and then reelecting the killed leader.
> 
> Here is an example log, which is representative of similar logs on the
other
> servers. We additionally logged connectivity during leader election.
If
> anyone would like complete logs, let me know.
> 
> Thanks,
> 
> Austin Shoemaker
> 
> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
> *WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
> ERROR - [QuorumPeer:Follower@137] - FIXMSG
> java.net.ConnectException: Connection refused
> *
> .... cont'd ....*
> 
> ERROR - [QuorumPeer:Follower@364] - FIXMSG
> java.lang.Exception: shutdown Follower
>         at
> com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java:364)
>         at
> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
> WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.22:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.22:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.21:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.21:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.12:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.12:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.11:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.11:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.12:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.12:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.11:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.11:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.22:2889
> *WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred
when
> sending / receiving packet to / from /10.50.65.22:2889
> java.net.SocketTimeoutException: Receive timed out
> *WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to
> /10.50.65.21:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.21:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.21:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.21:2889
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.12:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.12:2889
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election
packet to /
> 10.50.65.11:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from
/
> 10.50.65.11:2889
> WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
> WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
> WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
> WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
> WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
> result.winner = 7
> *WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete,
address
> = /10.50.65.22:2889
> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
> WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
> ERROR - [QuorumPeer:Follower@137] - FIXMSG
> java.net.ConnectException: Connection refused
> *        at java.net.PlainSocketImpl.socketConnect(Native Method)
>         at
java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
>         at
> java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>         at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>         at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>         at java.net.Socket.connect(Socket.java:519)
>         at
>
com.yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java:13
3)
>         at
> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)


Re: Leader election stalled

Posted by Austin Shoemaker <au...@cooliris.com>.
We killed the leader process, and it does not respond to ruok or  
election datagrams.

Austin

On Sep 2, 2008, at 10:05 AM, Mahadev Konar wrote:

> Hi Austin,
> Did you kill the leader process? It looks like that you didn't kill  
> the
> server since its responding to ruok. Is that true?
>
> mahadev
>
>
> On 9/2/08 9:56 AM, "Austin Shoemaker" <au...@cooliris.com> wrote:
>
>> Hi,
>>
>> We have run into a situation where killing the leader results in  
>> followers
>> perpetually trying to reelect that leader.
>>
>> We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients  
>> connecting
>> at random. We kill the leader and observe the impact, monitoring a  
>> script
>> that repeatedly prints the responses to "ruok" and "stat". All  
>> servers
>> except the killed leader respond with "imok" and "ZooKeeperServer not
>> running", respectively.
>>
>> About half of the time, each remaining server gets into a loop of  
>> failing to
>> connect to the killed leader and then reelecting the killed leader.
>>
>> Here is an example log, which is representative of similar logs on  
>> the other
>> servers. We additionally logged connectivity during leader  
>> election. If
>> anyone would like complete logs, let me know.
>>
>> Thanks,
>>
>> Austin Shoemaker
>>
>> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
>> *WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
>> ERROR - [QuorumPeer:Follower@137] - FIXMSG
>> java.net.ConnectException: Connection refused
>> *
>> .... cont'd ....*
>>
>> ERROR - [QuorumPeer:Follower@364] - FIXMSG
>> java.lang.Exception: shutdown Follower
>>        at
>> com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java: 
>> 364)
>>        at
>> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
>> WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.22:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.22:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.21:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.21:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.12:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.12:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.11:2888
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.11:2888
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.12:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.12:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.11:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.11:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.22:2889
>> *WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred  
>> when
>> sending / receiving packet to / from /10.50.65.22:2889
>> java.net.SocketTimeoutException: Receive timed out
>> *WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to
>> /10.50.65.21:2890
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.21:2890
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.21:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.21:2889
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.12:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.12:2889
>> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election  
>> packet to /
>> 10.50.65.11:2889
>> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response  
>> from /
>> 10.50.65.11:2889
>> WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
>> WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
>> WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
>> WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
>> WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
>> result.winner = 7
>> *WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete,  
>> address
>> = /10.50.65.22:2889
>> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
>> WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
>> ERROR - [QuorumPeer:Follower@137] - FIXMSG
>> java.net.ConnectException: Connection refused
>> *        at java.net.PlainSocketImpl.socketConnect(Native Method)
>>        at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java: 
>> 333)
>>        at
>> java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>>        at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>>        at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>>        at java.net.Socket.connect(Socket.java:519)
>>        at
>> com 
>> .yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java: 
>> 133)
>>        at
>> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)
>


Re: Leader election stalled

Posted by Mahadev Konar <ma...@yahoo-inc.com>.
Hi Austin,
 Did you kill the leader process? It looks like that you didn't kill the
server since its responding to ruok. Is that true?

mahadev


On 9/2/08 9:56 AM, "Austin Shoemaker" <au...@cooliris.com> wrote:

> Hi,
> 
> We have run into a situation where killing the leader results in followers
> perpetually trying to reelect that leader.
> 
> We have 11 zookeeper (2.2.1 from SF.net) servers and 256 clients connecting
> at random. We kill the leader and observe the impact, monitoring a script
> that repeatedly prints the responses to "ruok" and "stat". All servers
> except the killed leader respond with "imok" and "ZooKeeperServer not
> running", respectively.
> 
> About half of the time, each remaining server gets into a loop of failing to
> connect to the killed leader and then reelecting the killed leader.
> 
> Here is an example log, which is representative of similar logs on the other
> servers. We additionally logged connectivity during leader election. If
> anyone would like complete logs, let me know.
> 
> Thanks,
> 
> Austin Shoemaker
> 
> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
> *WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889*
> ERROR - [QuorumPeer:Follower@137] - FIXMSG
> java.net.ConnectException: Connection refused
> *
> .... cont'd ....*
> 
> ERROR - [QuorumPeer:Follower@364] - FIXMSG
> java.lang.Exception: shutdown Follower
>         at
> com.yahoo.zookeeper.server.quorum.Follower.shutdown(Follower.java:364)
>         at
> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:403)
> WARN  - [QuorumPeer:QuorumPeer@388] - LOOKING
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.22:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.22:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.21:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.21:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.12:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.12:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.11:2888
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.11:2888
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.12:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.12:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.11:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.11:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.22:2889
> *WARN  - [QuorumPeer:LeaderElection@166] - ----> Exception occurred when
> sending / receiving packet to / from /10.50.65.22:2889
> java.net.SocketTimeoutException: Receive timed out
> *WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to
> /10.50.65.21:2890
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.21:2890
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.21:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.21:2889
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.12:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.12:2889
> WARN  - [QuorumPeer:LeaderElection@136] - ----> Sending election packet to /
> 10.50.65.11:2889
> WARN  - [QuorumPeer:LeaderElection@153] - ----> Received response from /
> 10.50.65.11:2889
> WARN  - [QuorumPeer:LeaderElection@89] - Election tally:
> WARN  - [QuorumPeer:LeaderElection@95] - 8 -> 1
> WARN  - [QuorumPeer:LeaderElection@95] - 4 -> 1
> WARN  - [QuorumPeer:LeaderElection@95] - 7 -> 8
> WARN  - [QuorumPeer:LeaderElection@97] - ----> Election complete,
> result.winner = 7
> *WARN  - [QuorumPeer:LeaderElection@100] - ----> Election complete, address
> = /10.50.65.22:2889
> WARN  - [QuorumPeer:QuorumPeer@397] - FOLLOWING
> WARN  - [QuorumPeer:Follower@124] - Following /10.50.65.22:2889
> ERROR - [QuorumPeer:Follower@137] - FIXMSG
> java.net.ConnectException: Connection refused
> *        at java.net.PlainSocketImpl.socketConnect(Native Method)
>         at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:333)
>         at
> java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:195)
>         at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:182)
>         at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:366)
>         at java.net.Socket.connect(Socket.java:519)
>         at
> com.yahoo.zookeeper.server.quorum.Follower.followLeader(Follower.java:133)
>         at
> com.yahoo.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:399)