[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-11-21 Thread Michael K. Edwards (JIRA)


[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16695435#comment-16695435
 ] 

Michael K. Edwards commented on ZOOKEEPER-2930:
---

Fix needed for 3.5.5?

> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.5.3, 3.4.11, 3.5.4, 3.4.12
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
>  Labels: pull-request-available
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in the sendqueue of WorkerSender). At 
> last, the potential leader ofs_zk3 fails to receive the epoch ack in timeout, 
> so it quits the leader and begins a new election. 
> The log files of ofs_zk1 and ofs_zk3 are attached.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-03-07 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16389384#comment-16389384
 ] 

ASF GitHub Bot commented on ZOOKEEPER-2930:
---

Github user JonathanO commented on a diff in the pull request:

https://github.com/apache/zookeeper/pull/456#discussion_r172802731
  
--- Diff: 
src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java ---
@@ -318,76 +318,167 @@ public Thread newThread(Runnable r) {
  */
 public void testInitiateConnection(long sid) throws Exception {
 LOG.debug("Opening channel to server " + sid);
-Socket sock = new Socket();
-setSockOpts(sock);
-sock.connect(self.getVotingView().get(sid).electionAddr, cnxTO);
-initiateConnection(sock, sid);
+initiateConnection(sid, 
self.getVotingView().get(sid).electionAddr);
+}
+
+private Socket openChannel(long sid, InetSocketAddress electionAddr) {
+LOG.debug("Opening channel to server " + sid);
+try {
+final Socket sock = new Socket();
+setSockOpts(sock);
+sock.connect(electionAddr, cnxTO);
+LOG.debug("Connected to server " + sid);
+return sock;
+} catch (UnresolvedAddressException e) {
+// Sun doesn't include the address that causes this
+// exception to be thrown, also UAE cannot be wrapped cleanly
+// so we log the exception in order to capture this critical
+// detail.
+LOG.warn("Cannot open channel to " + sid
++ " at election address " + electionAddr, e);
+throw e;
+} catch (IOException e) {
+LOG.warn("Cannot open channel to " + sid
++ " at election address " + electionAddr,
+e);
+return null;
+}
 }
 
 /**
  * If this server has initiated the connection, then it gives up on the
  * connection if it loses challenge. Otherwise, it keeps the 
connection.
  */
-public void initiateConnection(final Socket sock, final Long sid) {
+public boolean initiateConnection(final Long sid, InetSocketAddress 
electionAddr) {
 try {
-startConnection(sock, sid);
-} catch (IOException e) {
-LOG.error("Exception while connecting, id: {}, addr: {}, 
closing learner connection",
-new Object[] { sid, sock.getRemoteSocketAddress() }, 
e);
-closeSocket(sock);
-return;
+Socket sock = openChannel(sid, electionAddr);
+if (sock != null) {
+try {
+startConnection(sock, sid);
+} catch (IOException e) {
+LOG.error("Exception while connecting, id: {}, addr: 
{}, closing learner connection",
+new Object[]{sid, 
sock.getRemoteSocketAddress()}, e);
+closeSocket(sock);
+}
+return true;
+} else {
+return false;
+}
+} finally {
+inprogressConnections.remove(sid);
 }
 }
 
-/**
- * Server will initiate the connection request to its peer server
- * asynchronously via separate connection thread.
- */
-public void initiateConnectionAsync(final Socket sock, final Long sid) 
{
+synchronized private void connectOneAsync(final Long sid, final 
ZooKeeperThread connectorThread) {
+if (senderWorkerMap.get(sid) != null) {
+LOG.debug("There is a connection already for server " + sid);
+return;
+}
 if(!inprogressConnections.add(sid)){
 // simply return as there is a connection request to
 // server 'sid' already in progress.
 LOG.debug("Connection request to server id: {} is already in 
progress, so skipping this request",
 sid);
-closeSocket(sock);
 return;
 }
 try {
-connectionExecutor.execute(
-new QuorumConnectionReqThread(sock, sid));
+connectionExecutor.execute(connectorThread);
 connectionThreadCnt.incrementAndGet();
 } catch (Throwable e) {
 // Imp: Safer side catching all type of exceptions and remove 
'sid'
 // from inprogress connections. This is to avoid blocking 
further
 // connection requests from this 'sid' in case of errors.
 

[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-03-05 Thread Jonathan Oddy (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16385920#comment-16385920
 ] 

Jonathan Oddy commented on ZOOKEEPER-2930:
--

PR #465 contains the backport of my fix to 3.4.x.

> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.5.3, 3.4.11, 3.5.4, 3.4.12
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in the sendqueue of WorkerSender). At 
> last, the potential leader ofs_zk3 fails to receive the epoch ack in timeout, 
> so it quits the leader and begins a new election. 
> The log files of ofs_zk1 and ofs_zk3 are attached.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-02-28 Thread Jiafu Jiang (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16381347#comment-16381347
 ] 

Jiafu Jiang commented on ZOOKEEPER-2930:


I hope this PB can be fix in version 3.4.X, since 3.4.X is the stable version.

> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.5.3, 3.4.11, 3.5.4, 3.4.12
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in the sendqueue of WorkerSender). At 
> last, the potential leader ofs_zk3 fails to receive the epoch ack in timeout, 
> so it quits the leader and begins a new election. 
> The log files of ofs_zk1 and ofs_zk3 are attached.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-02-22 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16372624#comment-16372624
 ] 

ASF GitHub Bot commented on ZOOKEEPER-2930:
---

Github user JonathanO commented on a diff in the pull request:

https://github.com/apache/zookeeper/pull/456#discussion_r169910520
  
--- Diff: 
src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java ---
@@ -318,76 +318,167 @@ public Thread newThread(Runnable r) {
  */
 public void testInitiateConnection(long sid) throws Exception {
 LOG.debug("Opening channel to server " + sid);
-Socket sock = new Socket();
-setSockOpts(sock);
-sock.connect(self.getVotingView().get(sid).electionAddr, cnxTO);
-initiateConnection(sock, sid);
+initiateConnection(sid, 
self.getVotingView().get(sid).electionAddr);
+}
+
+private Socket openChannel(long sid, InetSocketAddress electionAddr) {
+LOG.debug("Opening channel to server " + sid);
+try {
+final Socket sock = new Socket();
+setSockOpts(sock);
+sock.connect(electionAddr, cnxTO);
+LOG.debug("Connected to server " + sid);
+return sock;
+} catch (UnresolvedAddressException e) {
+// Sun doesn't include the address that causes this
+// exception to be thrown, also UAE cannot be wrapped cleanly
+// so we log the exception in order to capture this critical
+// detail.
+LOG.warn("Cannot open channel to " + sid
++ " at election address " + electionAddr, e);
+throw e;
+} catch (IOException e) {
+LOG.warn("Cannot open channel to " + sid
++ " at election address " + electionAddr,
+e);
+return null;
+}
 }
 
 /**
  * If this server has initiated the connection, then it gives up on the
  * connection if it loses challenge. Otherwise, it keeps the 
connection.
  */
-public void initiateConnection(final Socket sock, final Long sid) {
+public boolean initiateConnection(final Long sid, InetSocketAddress 
electionAddr) {
 try {
-startConnection(sock, sid);
-} catch (IOException e) {
-LOG.error("Exception while connecting, id: {}, addr: {}, 
closing learner connection",
-new Object[] { sid, sock.getRemoteSocketAddress() }, 
e);
-closeSocket(sock);
-return;
+Socket sock = openChannel(sid, electionAddr);
+if (sock != null) {
+try {
+startConnection(sock, sid);
+} catch (IOException e) {
+LOG.error("Exception while connecting, id: {}, addr: 
{}, closing learner connection",
+new Object[]{sid, 
sock.getRemoteSocketAddress()}, e);
+closeSocket(sock);
+}
+return true;
+} else {
+return false;
+}
+} finally {
+inprogressConnections.remove(sid);
 }
 }
 
-/**
- * Server will initiate the connection request to its peer server
- * asynchronously via separate connection thread.
- */
-public void initiateConnectionAsync(final Socket sock, final Long sid) 
{
+synchronized private void connectOneAsync(final Long sid, final 
ZooKeeperThread connectorThread) {
+if (senderWorkerMap.get(sid) != null) {
+LOG.debug("There is a connection already for server " + sid);
+return;
+}
 if(!inprogressConnections.add(sid)){
 // simply return as there is a connection request to
 // server 'sid' already in progress.
 LOG.debug("Connection request to server id: {} is already in 
progress, so skipping this request",
 sid);
-closeSocket(sock);
 return;
 }
 try {
-connectionExecutor.execute(
-new QuorumConnectionReqThread(sock, sid));
+connectionExecutor.execute(connectorThread);
 connectionThreadCnt.incrementAndGet();
 } catch (Throwable e) {
 // Imp: Safer side catching all type of exceptions and remove 
'sid'
 // from inprogress connections. This is to avoid blocking 
further
 // connection requests from this 'sid' in case of errors.
 

[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-02-15 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16366335#comment-16366335
 ] 

ASF GitHub Bot commented on ZOOKEEPER-2930:
---

Github user fpj commented on the issue:

https://github.com/apache/zookeeper/pull/456
  
I have also tested this PR a bit locally.


> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.5.3, 3.4.11, 3.5.4, 3.4.12
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in the sendqueue of WorkerSender). At 
> last, the potential leader ofs_zk3 fails to receive the epoch ack in timeout, 
> so it quits the leader and begins a new election. 
> The log files of ofs_zk1 and ofs_zk3 are attached.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-02-15 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16366332#comment-16366332
 ] 

ASF GitHub Bot commented on ZOOKEEPER-2930:
---

Github user fpj commented on a diff in the pull request:

https://github.com/apache/zookeeper/pull/456#discussion_r168620297
  
--- Diff: 
src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java ---
@@ -318,76 +318,167 @@ public Thread newThread(Runnable r) {
  */
 public void testInitiateConnection(long sid) throws Exception {
 LOG.debug("Opening channel to server " + sid);
-Socket sock = new Socket();
-setSockOpts(sock);
-sock.connect(self.getVotingView().get(sid).electionAddr, cnxTO);
-initiateConnection(sock, sid);
+initiateConnection(sid, 
self.getVotingView().get(sid).electionAddr);
+}
+
+private Socket openChannel(long sid, InetSocketAddress electionAddr) {
+LOG.debug("Opening channel to server " + sid);
+try {
+final Socket sock = new Socket();
+setSockOpts(sock);
+sock.connect(electionAddr, cnxTO);
+LOG.debug("Connected to server " + sid);
+return sock;
+} catch (UnresolvedAddressException e) {
+// Sun doesn't include the address that causes this
+// exception to be thrown, also UAE cannot be wrapped cleanly
+// so we log the exception in order to capture this critical
+// detail.
+LOG.warn("Cannot open channel to " + sid
++ " at election address " + electionAddr, e);
+throw e;
+} catch (IOException e) {
+LOG.warn("Cannot open channel to " + sid
++ " at election address " + electionAddr,
+e);
+return null;
+}
 }
 
 /**
  * If this server has initiated the connection, then it gives up on the
  * connection if it loses challenge. Otherwise, it keeps the 
connection.
  */
-public void initiateConnection(final Socket sock, final Long sid) {
+public boolean initiateConnection(final Long sid, InetSocketAddress 
electionAddr) {
 try {
-startConnection(sock, sid);
-} catch (IOException e) {
-LOG.error("Exception while connecting, id: {}, addr: {}, 
closing learner connection",
-new Object[] { sid, sock.getRemoteSocketAddress() }, 
e);
-closeSocket(sock);
-return;
+Socket sock = openChannel(sid, electionAddr);
+if (sock != null) {
+try {
+startConnection(sock, sid);
+} catch (IOException e) {
+LOG.error("Exception while connecting, id: {}, addr: 
{}, closing learner connection",
+new Object[]{sid, 
sock.getRemoteSocketAddress()}, e);
+closeSocket(sock);
+}
+return true;
+} else {
+return false;
+}
+} finally {
+inprogressConnections.remove(sid);
 }
 }
 
-/**
- * Server will initiate the connection request to its peer server
- * asynchronously via separate connection thread.
- */
-public void initiateConnectionAsync(final Socket sock, final Long sid) 
{
+synchronized private void connectOneAsync(final Long sid, final 
ZooKeeperThread connectorThread) {
+if (senderWorkerMap.get(sid) != null) {
+LOG.debug("There is a connection already for server " + sid);
+return;
+}
 if(!inprogressConnections.add(sid)){
 // simply return as there is a connection request to
 // server 'sid' already in progress.
 LOG.debug("Connection request to server id: {} is already in 
progress, so skipping this request",
 sid);
-closeSocket(sock);
 return;
 }
 try {
-connectionExecutor.execute(
-new QuorumConnectionReqThread(sock, sid));
+connectionExecutor.execute(connectorThread);
 connectionThreadCnt.incrementAndGet();
 } catch (Throwable e) {
 // Imp: Safer side catching all type of exceptions and remove 
'sid'
 // from inprogress connections. This is to avoid blocking 
further
 // connection requests from this 'sid' in case of errors.
 

[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-02-15 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16365494#comment-16365494
 ] 

ASF GitHub Bot commented on ZOOKEEPER-2930:
---

GitHub user JonathanO opened a pull request:

https://github.com/apache/zookeeper/pull/465

ZOOKEEPER-2930: Leader cannot be elected due to network timeout of some 
members.

Backported to 3.4 from the ZOOKEEPER-2930 branch.

Move sock.connect() into the async connection worker thread.
Moved a load of connectOne(sid) into the async connection worker thread.
Use use the async connection worker for all connections.
This prevents connection delays blocking notifications to other nodes.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/transferwise/zookeeper 
branch-3.4-ZOOKEEPER-2930

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/zookeeper/pull/465.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #465


commit ca05df6049e3282fef9fd3f34d07337011b5f4f6
Author: Jonathan Oddy 
Date:   2018-02-15T12:22:24Z

ZOOKEEPER-2930: Leader cannot be elected due to network timeout of some 
members.

Backported from 3.5.

Move sock.connect() into the async connection worker thread.
Moved a load of connectOne(sid) into the async connection worker thread.
Use use the async connection worker for all connections.
This prevents connection delays blocking notifications to other nodes.




> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.5.3, 3.4.11, 3.5.4, 3.4.12
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  

[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-02-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16352637#comment-16352637
 ] 

ASF GitHub Bot commented on ZOOKEEPER-2930:
---

Github user JonathanO commented on a diff in the pull request:

https://github.com/apache/zookeeper/pull/456#discussion_r166036745
  
--- Diff: 
src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java ---
@@ -689,15 +669,15 @@ synchronized void connectOne(long sid){
 Map lastProposedView = 
lastSeenQV.getAllMembers();
 if (lastCommittedView.containsKey(sid)) {
 knownId = true;
-if (connectOne(sid, 
lastCommittedView.get(sid).electionAddr))
-return;
--- End diff --

This part of the change isn't quite right since it relied on connectOne 
returning false on an IOException calling sock.connect(). We will no longer 
attempt to use lastProposedView.get(sid).electionAddr in the case that a 
connection using the lastCommittedView failed and the electionAddr has changed. 
I don't know what effect this will have. Maybe I need to move this condition 
into the async connection mechanism too?


> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.5.3, 3.4.11, 3.5.4, 3.4.12
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in the sendqueue of WorkerSender). At 
> last, the potential leader ofs_zk3 fails to 

[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-02-05 Thread Jonathan Oddy (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16352198#comment-16352198
 ] 

Jonathan Oddy commented on ZOOKEEPER-2930:
--

So, I think what happens is, if the 2nd node in the list dies in a way that 
causes new connections to time out then the notification messages to the 3rd 
node are delayed by >=5s while those to the 1st node are delivered on time. 
(sendNotifications() queues a notification to all three nodes (including the 
local node), in order, and toSend() blocks during sending the message to the 
2nd node.)

This 5s delay means that if the 3rd node is elected, it will see the election 
complete >= 5s after the 1st node does. The 1st node attempts to connect to the 
3rd on the leader port 5 times with a 1s delay (both hard coded) but, since the 
3rd node hasn't seen the election complete, it hasn't started listening on that 
port yet. Unless you're very lucky with timing, the 1st node will give up and 
start a new election round before the 3rd realises that it has been elected. 
The 3rd node then sits there for initLimit before going back to the LOOKING 
state, leaving you with a broken cluster for at least initLimit.

My patch attempts to fix this by making the entire process of establishing a 
connection async, avoiding it blocking toSend().

> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.5.3, 3.4.11, 3.5.4, 3.4.12
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in 

[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2018-02-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16350736#comment-16350736
 ] 

ASF GitHub Bot commented on ZOOKEEPER-2930:
---

GitHub user JonathanO opened a pull request:

https://github.com/apache/zookeeper/pull/456

ZOOKEEPER-2930: Leader cannot be elected due to network timeout of some 
members.

Move sock.connect() into the async connection worker thread.
Use initiateConnectionAsync for all connections.
This prevents connection delays blocking notifications to other nodes.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/transferwise/zookeeper ZOOKEEPER-2930

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/zookeeper/pull/456.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #456


commit 68e327e112b0910e56b9fa2b3ba948e4179adb6b
Author: Jonathan Oddy 
Date:   2018-02-02T15:33:50Z

ZOOKEEPER-2930: Leader cannot be elected due to network timeout of some 
members.

Move sock.connect() into the async connection worker thread.
Use initiateConnectionAsync for all connections.
This prevents connection delays blocking notifications to other nodes.




> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.5.3, 3.4.11, 3.5.4, 3.4.12
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the 

[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2017-11-12 Thread caixiaofeng (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16249027#comment-16249027
 ] 

caixiaofeng commented on ZOOKEEPER-2930:


 Mark, meet the same problem.

> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10, 3.4.11
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Critical
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in the sendqueue of WorkerSender). At 
> last, the potential leader ofs_zk3 fails to receive the epoch ack in timeout, 
> so it quits the leader and begins a new election. 
> The log files of ofs_zk1 and ofs_zk3 are attached.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2017-11-07 Thread yangzhixin (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16241912#comment-16241912
 ] 

yangzhixin commented on ZOOKEEPER-2930:
---

I encounter the same problem. Wait for a conclusion.

> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection, quorum, server
>Affects Versions: 3.4.10
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in the sendqueue of WorkerSender). At 
> last, the potential leader ofs_zk3 fails to receive the epoch ack in timeout, 
> so it quits the leader and begins a new election. 
> The log files of ofs_zk1 and ofs_zk3 are attached.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (ZOOKEEPER-2930) Leader cannot be elected due to network timeout of some members.

2017-11-04 Thread Jiafu Jiang (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16238828#comment-16238828
 ] 

Jiafu Jiang commented on ZOOKEEPER-2930:


I suggest that there can be more that one WorkSender in FastLeaderElection, so 
that network failure of some zk servers will not affect the notifications to 
others.

> Leader cannot be elected due to network timeout of some members.
> 
>
> Key: ZOOKEEPER-2930
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2930
> Project: ZooKeeper
>  Issue Type: Bug
>  Components: leaderElection
>Affects Versions: 3.4.10
> Environment: Java 8
> ZooKeeper 3.4.11(from github)
> Centos6.5
>Reporter: Jiafu Jiang
>Priority: Major
> Attachments: zoo.cfg, zookeeper1.log, zookeeper2.log
>
>
> I deploy a cluster of ZooKeeper with three nodes:
> ofs_zk1:20.10.11.101, 30.10.11.101
> ofs_zk2:20.10.11.102, 30.10.11.102
> ofs_zk3:20.10.11.103, 30.10.11.103
> I shutdown the network interfaces of ofs_zk2 using "ifdown eth0 eth1" command.
> It is supposed that the new Leader should be elected in some seconds, but the 
> fact is, ofs_zk1 and ofs_zk3 just keep electing again and again, but none of 
> them can become the new Leader.
> I change the log level to DEBUG (the default is INFO), and restart zookeeper 
> servers on ofs_zk1 and ofs_zk2 again, but it can not fix the problem.
> I read the log and the ZooKeeper source code, and I think I find the reason.
> When the potential leader(says ofs_zk3) begins the 
> election(FastLeaderElection.lookForLeader()), it will send notifications to 
> all the servers. 
> When it fails to receive any notification during a timeout, it will resend 
> the notifications, and double the timeout. This process will repeat until any 
> notification is received or the timeout reaches a max value.
> The FastLeaderElection.sendNotifications() just put the notification message 
> into a queue and return. The WorkerSender is responsable to send the 
> notifications.
> The WorkerSender just process the notifications one by one by passing the 
> notifications to QuorumCnxManager. Here comes the problem, the 
> QuorumCnxManager.toSend() blocks for a long time when the notification is 
> send to ofs_zk2(whose network is down) and some notifications (which belongs 
> to ofs_zk1) will thus be blocked for a long time. The repeated notifications 
> by FastLeaderElection.sendNotifications() just make things worse.
> Here is the related source code:
> {code:java}
> public void toSend(Long sid, ByteBuffer b) {
> /*
>  * If sending message to myself, then simply enqueue it (loopback).
>  */
> if (this.mySid == sid) {
>  b.position(0);
>  addToRecvQueue(new Message(b.duplicate(), sid));
> /*
>  * Otherwise send to the corresponding thread to send.
>  */
> } else {
>  /*
>   * Start a new connection if doesn't have one already.
>   */
>  ArrayBlockingQueue bq = new 
> ArrayBlockingQueue(SEND_CAPACITY);
>  ArrayBlockingQueue bqExisting = 
> queueSendMap.putIfAbsent(sid, bq);
>  if (bqExisting != null) {
>  addToSendQueue(bqExisting, b);
>  } else {
>  addToSendQueue(bq, b);
>  }
>  
>  // This may block!!!
>  connectOne(sid);
> 
> }
> }
> {code}
> Therefore, when ofs_zk3 believes that it is the leader, it begins to wait the 
> epoch ack, but in fact the ofs_zk1 does not receive the notification(which 
> says the leader is ofs_zk3) because the ofs_zk3 has not sent the 
> notification(which may still exist in the sendqueue of WorkerSender). At 
> last, the potential leader ofs_zk3 fails to receive the epoch ack in timeout, 
> so it quits the leader and begins a new election. 
> The log files of ofs_zk1 and ofs_zk3 are attached.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)