[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2386?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Enis Soztutar updated ZOOKEEPER-2386:
-------------------------------------
    Attachment: zklogs.tar.gz

Attaching relevant logs from two servers that are alive. As you can see, we 
keep cycling the leader election + timeout loop forever. zk2 is always the 
leader, and from that servers perspective each cycle of leader election takes 
15 secs, while zk0 reports it around 30 sec. 

Again, (I think) the issue is that the vote that the candidate does not send 
the votes in parallel, but serially. So zk2 as a candidate cannot cast a self 
vote until the request to zk1 timesouts which takes more than 5 seconds. By 
that time, the follower zk0 already gives up on the leader, and starts a new 
round. So there is actually a leader elected every ~30 seconds, but the quorum 
cannot operate still. 

zk0: 
{code}
2016-01-27 23:30:15,293 [myid:1] - INFO  
[QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2182:Follower@63] - FOLLOWING - LEADER 
ELECTION TOOK - 31008
2016-01-27 23:30:50,417 [myid:1] - INFO  
[QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2182:Follower@63] - FOLLOWING - LEADER 
ELECTION TOOK - 31112
2016-01-27 23:31:25,326 [myid:1] - INFO  
[QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2182:Follower@63] - FOLLOWING - LEADER 
ELECTION TOOK - 30898
2016-01-27 23:32:00,340 [myid:1] - INFO  
[QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2182:Follower@63] - FOLLOWING - LEADER 
ELECTION TOOK - 30997
2016-01-27 23:32:35,365 [myid:1] - INFO  
[QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2182:Follower@63] - FOLLOWING - LEADER 
ELECTION TOOK - 31014
2016-01-27 23:33:10,391 [myid:1] - INFO  
[QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2182:Follower@63] - FOLLOWING - LEADER 
ELECTION TOOK - 31014
2016-01-27 23:33:45,485 [myid:1] - INFO  
[QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2182:Follower@63] - FOLLOWING - LEADER 
ELECTION TOOK - 31082
{code}

zk2:
{code}
2016-01-27 23:30:20,080 [myid:3] - INFO  
[QuorumPeer[myid=3]/0:0:0:0:0:0:0:0:2182:Leader@358] - LEADING - LEADER 
ELECTION TOOK - 14816
2016-01-27 23:30:55,407 [myid:3] - INFO  
[QuorumPeer[myid=3]/0:0:0:0:0:0:0:0:2182:Leader@358] - LEADING - LEADER 
ELECTION TOOK - 15324
2016-01-27 23:31:30,314 [myid:3] - INFO  
[QuorumPeer[myid=3]/0:0:0:0:0:0:0:0:2182:Leader@358] - LEADING - LEADER 
ELECTION TOOK - 14904
2016-01-27 23:32:05,327 [myid:3] - INFO  
[QuorumPeer[myid=3]/0:0:0:0:0:0:0:0:2182:Leader@358] - LEADING - LEADER 
ELECTION TOOK - 15010
2016-01-27 23:32:40,353 [myid:3] - INFO  
[QuorumPeer[myid=3]/0:0:0:0:0:0:0:0:2182:Leader@358] - LEADING - LEADER 
ELECTION TOOK - 15023
2016-01-27 23:33:15,179 [myid:3] - INFO  
[QuorumPeer[myid=3]/0:0:0:0:0:0:0:0:2182:Leader@358] - LEADING - LEADER 
ELECTION TOOK - 14823
2016-01-27 23:33:50,476 [myid:3] - INFO  
[QuorumPeer[myid=3]/0:0:0:0:0:0:0:0:2182:Leader@358] - LEADING - LEADER 
ELECTION TOOK - 15294
2016-01-27 23:34:25,234 [myid:3] - INFO  
[QuorumPeer[myid=3]/0:0:0:0:0:0:0:0:2182:Leader@358] - LEADING - LEADER 
ELECTION TOOK - 14756
20
{code}


> Cannot achieve quorum when middle server (in a q of 3) is unreacable
> --------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-2386
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2386
>             Project: ZooKeeper
>          Issue Type: Bug
>            Reporter: Enis Soztutar
>         Attachments: zklogs.tar.gz
>
>
> Recently, we've observed a curious case where a quorum was not reached for 
> days in a cluster of 3 nodes (zk0, zk1, zk2) and the middle node zk1 is 
> unreachable from network. 
> The leader election happens, and both zk0 and zk2 starts the vote. Then each 
> server sends notifications to every other server including itself. The 
> problem is that, zk1 vm is unavailable, so when we are trying to open up a 
> socket to connect to that server with socket timeout of 5 seconds, it delays 
> the notification processing of the vote sent from zk2 to zk2 (itself). The 
> vote eventually comes after 5 sec, but by that time, the follower (zk0) 
> already converted to the follower state. On the follower state, the follower 
> try to connect to leader 5 times with 1 second timeout (5 sec in total). 
> Since the leader does not start its peer port for 5 seconds after the 
> follower starts, the follower always times out connecting to the leader. This 
> cycle is repeating for hours / days even after restarting the servers several 
> times. 
> I believe this is related to the default timeouts (5 sec socket timeout) and 
> follower to leader connection timeout (5 tries with 1 second timeout). Only 
> after setting the {{zookeeper.cnxTimeout}} to 1 second, the quorum was 
> operating. 
> More logs coming shortly. 
> zoo.cfg: 
> {code}
> server.3=zk2-hostname:2889:3889
> server.2=zk1-hostname:2889:3889
> server.1=zk0-hostname:2889:3889
> {code}



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

Reply via email to