[jira] [Commented] (ZOOKEEPER-2701) Timeout for RecvWorker is too long

2018-07-16 Thread Jiafu Jiang (JIRA)


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

Jiafu Jiang commented on ZOOKEEPER-2701:


I read the source code of  ZooKeeper 3.4.12, and I find the SendWorker or 
RecvWorker will be finished when an IOException  happends.

When network problems happen, the OS may or may not discover the dead 
connection in time,  especially when the socket timeout is infinity.  This will 
lead to problem that ZooKeeper take *several*  minutes to elect new leader.

 

 

> Timeout for RecvWorker is too long
> --
>
> Key: ZOOKEEPER-2701
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2701
> Project: ZooKeeper
>  Issue Type: Bug
>Affects Versions: 3.4.8, 3.4.9, 3.4.10, 3.4.11
> Environment: Centos6.5
> ZooKeeper 3.4.8
>Reporter: Jiafu Jiang
>Priority: Major
>
> Environment:
>  I deploy ZooKeeper in a cluster of three nodes. Each node has three network 
> interfaces(eth0, eth1, eth2).
> Hostname is used instead of IP address in zoo.cfg, and 
> quorumListenOnAllIPs=true
> Probleam:
>  I start three ZooKeeper servers( node A, node B, and node C) one by one, 
>  when the leader election finishes, node B is the leader. 
>  Then I shutdown one network interface of node A by command "ifdown eth0". 
> The ZooKeeper server on node A will lost connection to node B and node C. In 
> my test, I will take about 20 minites that the ZooKeepr server of node A 
> realizes the event and try to call the QuorumServer.recreateSocketAddress the 
> resolve the hostname.
> I try to read the source code, and I find the code in
> {code:java|title=QuorumCnxManager.java:|borderStyle=solid}
> class RecvWorker extends ZooKeeperThread {
> Long sid;
> Socket sock;
> volatile boolean running = true;
> final DataInputStream din;
> final SendWorker sw;
> RecvWorker(Socket sock, DataInputStream din, Long sid, SendWorker sw) 
> {
> super("RecvWorker:" + sid);
> this.sid = sid;
> this.sock = sock;
> this.sw = sw;
> this.din = din;
> try {
> // OK to wait until socket disconnects while reading.
> sock.setSoTimeout(0);
> } catch (IOException e) {
> LOG.error("Error while accessing socket for " + sid, e);
> closeSocket(sock);
> running = false;
> }
> }
>...
>  }
> {code}
> I notice that the soTime is set to 0 in RecvWorker constructor. I think this 
> is reasonable when the IP address of a ZooKeeper server never change, but 
> considering that the IP address of each ZooKeeper server may change, maybe we 
> should better set a timeout here.
> I think this is a problem.



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


[jira] [Commented] (ZOOKEEPER-2701) Timeout for RecvWorker is too long

2018-07-16 Thread Tom Lee (JIRA)


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

Tom Lee commented on ZOOKEEPER-2701:


I can't speak to whether that solution is correct (suspect it's likely more 
complicated than simply removing the code that modifies the read timeout for 
all sorts of reasons), but I can imagine how the read timeout firing may 
address some of the symptoms by forcing the socket closed & short-circuiting 
the OS-level retransmission guff.

Whatever the outcome, thanks for posting this bug! :) It was very helpful for 
tracking down what was going on & helped us nail down our own system-level 
work-around (the tcp_retries2 thing seems to work great)

> Timeout for RecvWorker is too long
> --
>
> Key: ZOOKEEPER-2701
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2701
> Project: ZooKeeper
>  Issue Type: Bug
>Affects Versions: 3.4.8, 3.4.9, 3.4.10, 3.4.11
> Environment: Centos6.5
> ZooKeeper 3.4.8
>Reporter: Jiafu Jiang
>Priority: Major
>
> Environment:
>  I deploy ZooKeeper in a cluster of three nodes. Each node has three network 
> interfaces(eth0, eth1, eth2).
> Hostname is used instead of IP address in zoo.cfg, and 
> quorumListenOnAllIPs=true
> Probleam:
>  I start three ZooKeeper servers( node A, node B, and node C) one by one, 
>  when the leader election finishes, node B is the leader. 
>  Then I shutdown one network interface of node A by command "ifdown eth0". 
> The ZooKeeper server on node A will lost connection to node B and node C. In 
> my test, I will take about 20 minites that the ZooKeepr server of node A 
> realizes the event and try to call the QuorumServer.recreateSocketAddress the 
> resolve the hostname.
> I try to read the source code, and I find the code in
> {code:java|title=QuorumCnxManager.java:|borderStyle=solid}
> class RecvWorker extends ZooKeeperThread {
> Long sid;
> Socket sock;
> volatile boolean running = true;
> final DataInputStream din;
> final SendWorker sw;
> RecvWorker(Socket sock, DataInputStream din, Long sid, SendWorker sw) 
> {
> super("RecvWorker:" + sid);
> this.sid = sid;
> this.sock = sock;
> this.sw = sw;
> this.din = din;
> try {
> // OK to wait until socket disconnects while reading.
> sock.setSoTimeout(0);
> } catch (IOException e) {
> LOG.error("Error while accessing socket for " + sid, e);
> closeSocket(sock);
> running = false;
> }
> }
>...
>  }
> {code}
> I notice that the soTime is set to 0 in RecvWorker constructor. I think this 
> is reasonable when the IP address of a ZooKeeper server never change, but 
> considering that the IP address of each ZooKeeper server may change, maybe we 
> should better set a timeout here.
> I think this is a problem.



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


[jira] [Commented] (ZOOKEEPER-2701) Timeout for RecvWorker is too long

2018-07-16 Thread Jiafu Jiang (JIRA)


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

Jiafu Jiang commented on ZOOKEEPER-2701:


I remove the following code:
try {// OK to wait until socket disconnects while 
reading.sock.setSoTimeout(0);
} catch (IOException e) {
LOG.error("Error while accessing socket for " + sid, e);
closeSocket(sock);
running = false;
}
 

And I find it works fine in my test environment.

> Timeout for RecvWorker is too long
> --
>
> Key: ZOOKEEPER-2701
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2701
> Project: ZooKeeper
>  Issue Type: Bug
>Affects Versions: 3.4.8, 3.4.9, 3.4.10, 3.4.11
> Environment: Centos6.5
> ZooKeeper 3.4.8
>Reporter: Jiafu Jiang
>Priority: Minor
>
> Environment:
>  I deploy ZooKeeper in a cluster of three nodes. Each node has three network 
> interfaces(eth0, eth1, eth2).
> Hostname is used instead of IP address in zoo.cfg, and 
> quorumListenOnAllIPs=true
> Probleam:
>  I start three ZooKeeper servers( node A, node B, and node C) one by one, 
>  when the leader election finishes, node B is the leader. 
>  Then I shutdown one network interface of node A by command "ifdown eth0". 
> The ZooKeeper server on node A will lost connection to node B and node C. In 
> my test, I will take about 20 minites that the ZooKeepr server of node A 
> realizes the event and try to call the QuorumServer.recreateSocketAddress the 
> resolve the hostname.
> I try to read the source code, and I find the code in
> {code:java|title=QuorumCnxManager.java:|borderStyle=solid}
> class RecvWorker extends ZooKeeperThread {
> Long sid;
> Socket sock;
> volatile boolean running = true;
> final DataInputStream din;
> final SendWorker sw;
> RecvWorker(Socket sock, DataInputStream din, Long sid, SendWorker sw) 
> {
> super("RecvWorker:" + sid);
> this.sid = sid;
> this.sock = sock;
> this.sw = sw;
> this.din = din;
> try {
> // OK to wait until socket disconnects while reading.
> sock.setSoTimeout(0);
> } catch (IOException e) {
> LOG.error("Error while accessing socket for " + sid, e);
> closeSocket(sock);
> running = false;
> }
> }
>...
>  }
> {code}
> I notice that the soTime is set to 0 in RecvWorker constructor. I think this 
> is reasonable when the IP address of a ZooKeeper server never change, but 
> considering that the IP address of each ZooKeeper server may change, maybe we 
> should better set a timeout here.
> I think this is a problem.



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


[jira] [Commented] (ZOOKEEPER-2701) Timeout for RecvWorker is too long

2018-07-16 Thread Jiafu Jiang (JIRA)


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

Jiafu Jiang commented on ZOOKEEPER-2701:


I remove the following code:
try {// OK to wait until socket disconnects while 
reading.sock.setSoTimeout(0);
} catch (IOException e) {
LOG.error("Error while accessing socket for " + sid, e);
closeSocket(sock);
running = false;
}

And I find it works fine in my test environment. 


> Timeout for RecvWorker is too long
> --
>
> Key: ZOOKEEPER-2701
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2701
> Project: ZooKeeper
>  Issue Type: Bug
>Affects Versions: 3.4.8, 3.4.9, 3.4.10, 3.4.11
> Environment: Centos6.5
> ZooKeeper 3.4.8
>Reporter: Jiafu Jiang
>Priority: Minor
>
> Environment:
>  I deploy ZooKeeper in a cluster of three nodes. Each node has three network 
> interfaces(eth0, eth1, eth2).
> Hostname is used instead of IP address in zoo.cfg, and 
> quorumListenOnAllIPs=true
> Probleam:
>  I start three ZooKeeper servers( node A, node B, and node C) one by one, 
>  when the leader election finishes, node B is the leader. 
>  Then I shutdown one network interface of node A by command "ifdown eth0". 
> The ZooKeeper server on node A will lost connection to node B and node C. In 
> my test, I will take about 20 minites that the ZooKeepr server of node A 
> realizes the event and try to call the QuorumServer.recreateSocketAddress the 
> resolve the hostname.
> I try to read the source code, and I find the code in
> {code:java|title=QuorumCnxManager.java:|borderStyle=solid}
> class RecvWorker extends ZooKeeperThread {
> Long sid;
> Socket sock;
> volatile boolean running = true;
> final DataInputStream din;
> final SendWorker sw;
> RecvWorker(Socket sock, DataInputStream din, Long sid, SendWorker sw) 
> {
> super("RecvWorker:" + sid);
> this.sid = sid;
> this.sock = sock;
> this.sw = sw;
> this.din = din;
> try {
> // OK to wait until socket disconnects while reading.
> sock.setSoTimeout(0);
> } catch (IOException e) {
> LOG.error("Error while accessing socket for " + sid, e);
> closeSocket(sock);
> running = false;
> }
> }
>...
>  }
> {code}
> I notice that the soTime is set to 0 in RecvWorker constructor. I think this 
> is reasonable when the IP address of a ZooKeeper server never change, but 
> considering that the IP address of each ZooKeeper server may change, maybe we 
> should better set a timeout here.
> I think this is a problem.



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


[jira] [Commented] (ZOOKEEPER-2701) Timeout for RecvWorker is too long

2018-06-26 Thread Tom Lee (JIRA)


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

Tom Lee commented on ZOOKEEPER-2701:


Not entirely sure our symptoms were exactly the same as [~jiangjiafu], but 
believe we were bitten by the `setSoTimeout(0)` thing too (we're running 
3.4.11). Still trying to work out the exact circumstances that cause the issue 
to occur, but I suspect it's something like:
 # a NIC intermittently drops out & recovers quickly on a follower.
 # #1 is enough to cause socket writes to fail in such a way that Linux packet 
retransmission + exponential backoff foo kicks.
 # the follower attempts to rejoin the cluster, but is unable to due to some 
combination of the above failing write(s) and rejecting new inbound connections 
from other members of the quorum because "There is already a connection for 
server X".
 # roughly 15-16 minutes later, Linux detects the connection is dead after 
exhausting packet retransmission retries & forces the connections closed.
 # #4 allows new inbound connections, socket writes start succeeding, the 
follower rejoins the quorum and everything is great again.

This prevents the follower from rejoining the quorum, which obviously isn't 
great. Given that, this is a little worse than a "Minor" issue when it happens 
IMO – if you're unlucky this could happen across multiple machines at the same 
time & easily lead to quorum loss.

I may not have the exact details down, but it's _something_ like that. This 
issue has been occurring regularly across multiple clusters we've got running 
in EC2.

If it's useful to anybody playing along at home: we're experimenting with 
[tcp_retries2|https://www.kernel.org/doc/Documentation/networking/ip-sysctl.txt]
 to work around the issue at a system level. The default behavior in our setup 
(which from a quick google seems to be pretty standard) is to retry 15 times 
with up to a 2 minute delay between retries for a total of 15-16 minutes – 
seems safe to be a bit more aggressive given our ZK hosts are all in the same 
EC2 region. I've seen some suggestions of folks going with 3 retries, but we're 
being a bit more conservative & going with 5 for now, which should let us 
detect dead connections within a few seconds.

 

> Timeout for RecvWorker is too long
> --
>
> Key: ZOOKEEPER-2701
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2701
> Project: ZooKeeper
>  Issue Type: Bug
>Affects Versions: 3.4.8, 3.4.9, 3.4.10
> Environment: Centos6.5
> ZooKeeper 3.4.8
>Reporter: Jiafu Jiang
>Priority: Minor
>
> Environment:
> I deploy ZooKeeper in a cluster of three nodes. Each node has three network 
> interfaces(eth0, eth1, eth2).
> Hostname is used instead of IP address in zoo.cfg, and 
> quorumListenOnAllIPs=true
> Probleam:
> I start three ZooKeeper servers( node A, node B, and node C) one by one, 
> when the leader election finishes, node B is the leader. 
> Then I shutdown one network interface of node A by command "ifdown eth0". The 
> ZooKeeper server on node A will lost connection to node B and node C. In my 
> test, I will take about 20 minites that the ZooKeepr server of node A 
> realizes the event and try to call the QuorumServer.recreateSocketAddress the 
> resolve the hostname.
> I try to read the source code, and I find the code in 
> {code:title=QuorumCnxManager.java:|borderStyle=solid}
> class RecvWorker extends ZooKeeperThread {
> Long sid;
> Socket sock;
> volatile boolean running = true;
> final DataInputStream din;
> final SendWorker sw;
> RecvWorker(Socket sock, DataInputStream din, Long sid, SendWorker sw) 
> {
> super("RecvWorker:" + sid);
> this.sid = sid;
> this.sock = sock;
> this.sw = sw;
> this.din = din;
> try {
> // OK to wait until socket disconnects while reading.
> sock.setSoTimeout(0);
> } catch (IOException e) {
> LOG.error("Error while accessing socket for " + sid, e);
> closeSocket(sock);
> running = false;
> }
> }
>...
>  }
> {code}
> I notice that the soTime is set to 0 in RecvWorker constructor. I think this 
> is reasonable when the IP address of a ZooKeeper server never change, but  
> considering that the IP address of each ZooKeeper server may change, maybe we 
> should better set a timeout here.
> I am not pretty sure this is really a problem. 



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