You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by "Jiafu Jiang (JIRA)" <ji...@apache.org> on 2018/07/16 07:30:00 UTC

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

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

Jiafu Jiang updated ZOOKEEPER-2701:
-----------------------------------
    Description: 
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.

  was:
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. 




> 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: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)