You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@zookeeper.apache.org by GitBox <gi...@apache.org> on 2020/01/25 08:38:24 UTC

[GitHub] [zookeeper] yfxhust opened a new pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

yfxhust opened a new pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235
 
 
   …ork is broken
   
   - add unit test to verify the bug
   - bypass the SendThread.startConnect() by throw RuntimeExcepth if state.isAlive is false
   
   Author: Fangxi Yin <yi...@kuaishou.com>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379822759
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
         }
 
+        synchronized void setZkState(ZooKeeper.States newState) throws IOException {
+            if (!state.isAlive()) {
+                if (newState == States.CONNECTING) {
+                    throw new IOException(
 
 Review comment:
   `if (!state.isAlive() && newState == States.CONNECTING)` ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] eolivelli commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r385524199
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -957,7 +962,11 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
 
         SendThread(ClientCnxnSocket clientCnxnSocket) {
             super(makeThreadName("-SendThread()"));
-            state = States.CONNECTING;
+            try {
+                changeZkState(States.CONNECTING);
+            } catch (IOException e) {
+                throw new RuntimeException("Connection setup failed when migrate state to CONNECTING");
 
 Review comment:
   Throwing a RuntimeException may lead to unexpected behaviour. 
   So we have a way to throw a checked exception and deal explicitly with this case?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373914530
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -782,6 +782,10 @@ void queueCallback(AsyncCallback cb, int rc, String path, Object ctx) {
         eventThread.queueCallback(cb, rc, path, ctx);
     }
 
+    protected void onConnecting(InetSocketAddress addr) {
+        LOG.info("Connecting server {}.", addr);
+    }
+
 
 Review comment:
   Actually onConnecting() method is added to facilitate the test case. onConnecting() is the unsafe point that is described in JIRA issue. We can override this method to inject some actions in test case. The injected actions make issue happen. If we don't have this method, it is difficult to construct a test case to reproduce the issue.
   Your suggestion of log changing is good for me. Accept it. I will change the log to "Try to reconnect to server {}." Thanks for your comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373914530
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -782,6 +782,10 @@ void queueCallback(AsyncCallback cb, int rc, String path, Object ctx) {
         eventThread.queueCallback(cb, rc, path, ctx);
     }
 
+    protected void onConnecting(InetSocketAddress addr) {
+        LOG.info("Connecting server {}.", addr);
+    }
+
 
 Review comment:
   Actually onConnecting() method is added to facilitate the test case. onConnecting() is the unsafe point that is described in JIRA issue. We can override this method to inject some actions in test case. The injected actions make test case happen. If we don't have this method, it is difficult to construct a test case to reproduce the issue.
   Your suggestion of log changing is good for me. Accept it. I will change the log to "Try to reconnect to server {}." Thanks for your comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r374381726
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -782,6 +782,10 @@ void queueCallback(AsyncCallback cb, int rc, String path, Object ctx) {
         eventThread.queueCallback(cb, rc, path, ctx);
     }
 
+    protected void onConnecting(InetSocketAddress addr) {
+        LOG.info("Try to connect to server {}.", addr);
 
 Review comment:
   Since the primary reason of adding this is for injection in unit tests, I'd suggest we mark this method which a comment like "for tests only", and remove the log statement which if leaving as is, will create a lot of noises in client side logs. We can still override an empty method in test implementation.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r374466290
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -782,6 +782,10 @@ void queueCallback(AsyncCallback cb, int rc, String path, Object ctx) {
         eventThread.queueCallback(cb, rc, path, ctx);
     }
 
+    protected void onConnecting(InetSocketAddress addr) {
+        LOG.info("Try to connect to server {}.", addr);
 
 Review comment:
   Thanks your comments. Accept it. I will add comments to declare it as "for test only" and leave this method as empty.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r386179391
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -957,7 +962,11 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
 
         SendThread(ClientCnxnSocket clientCnxnSocket) {
             super(makeThreadName("-SendThread()"));
-            state = States.CONNECTING;
+            try {
+                changeZkState(States.CONNECTING);
+            } catch (IOException e) {
+                throw new RuntimeException("Connection setup failed when migrate state to CONNECTING");
 
 Review comment:
   Thanks you for your comments. I accept your suggestion. I throw checked IOException for SendThread constructor. Please review latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] eolivelli commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r385524922
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,333 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.ClientCnxn.Packet;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ClientCnxnSocketFragilityTest extends QuorumPeerTestBase {
+
+    private static final int SERVER_COUNT = 3;
+
+    private static final int SESSION_TIMEOUT = 40000;
+
+    public static final int CONNECTION_TIMEOUT = 30000;
+
+    private final UnsafeCoordinator unsafeCoordinator = new UnsafeCoordinator();
+
+    private volatile CustomZooKeeper zk = null;
+
+    private volatile FragileClientCnxnSocketNIO socket = null;
+
+    private volatile CustomClientCnxn cnxn = null;
+
+    private String getCxnString(int[] clientPorts) {
+        StringBuffer hostPortBuffer = new StringBuffer();
+        for (int i = 0; i < clientPorts.length; i++) {
+            hostPortBuffer.append("127.0.0.1:");
+            hostPortBuffer.append(clientPorts[i]);
+            if (i != (clientPorts.length - 1)) {
+                hostPortBuffer.append(',');
+            }
+        }
+        return hostPortBuffer.toString();
+    }
+
+    private void closeZookeeper(ZooKeeper zk) {
+        Executors.newSingleThreadExecutor().submit(() -> {
+            try {
+                LOG.info("closeZookeeper is fired");
+                zk.close();
+            } catch (InterruptedException e) {
+            }
+        });
+    }
+
+    private void getDataBackgroundRetryForever(CustomZooKeeper zk, String path) {
+        new Thread(() -> {
+            for (;;) {
 
 Review comment:
   Please use a daemon thread.
   I wonder of we can create a safer way to stop this thread.
   I wouldn't like to leak this thread during the execution of the test suite.
   It could easily use a CPU at 100%

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r374471742
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
 
 Review comment:
   synchronized(state) is still not safe. Maybe SendThread.close() and SendThread.startConnect() synchronize on different objects because state is mutable and can be changed between SendThread.close() and SendThread.startConnect().
   I prefer synchronized(this) guard to lock on SendThread..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-590761085
 
 
   @hanm Any further comments ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379970390
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   Thank you for your comments.
   spotbugs-maven-plugin:3.1.9:check may report error for UG_SYNC_SET_UNSYNC_GET if remove `synchronized` from getZkState because we have synchronized setZkState. 
   Another reason of adding `synchronized` is to prevent future risk if we add more complex task in setZkState() in future.
   In current implementation, we have implicit convention for setZkSate() - state modification must be placed at the end of setZkState() otherwise incomplete internal state may be exposed to outside. Adding `synchronized` to getZkState() can avoid this risk because getZkState() only be allowed to interleave execution with the whole setZkState().
   
   Anyway I accept your suggestion. Maybe it has potential performance benefit if I remove `synchronized` from getZkState(). I will add more comments to clarify the implicit convention of state modification and rename setZkState() to changZkState() to make spotbugs-maven-plugin happy.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379970390
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   Thank you for your comments.
   In my memory, checkstyle may report exception if remove `synchronized` from getkState. But I can not reproduce the checkstyle error in current version. Maybe I missed something. 
   Another reason of adding `synchronized` is to prevent future risk if we add more complex task in setZkState() in future..
   In current implementation, we have implicit convention for setZkSate() - state modification must be placed at the end of setZkState() otherwise incomplete internal state may be exposed to outside. Adding `synchronized` to getZkState() can avoid this risk because getZkState() only be allowed to interleave execution with the whole setZkState().
   
   Anyway I accept your suggestion. Maybe it has potential performance benefit if I remove `synchronized` from getZkState(). I will add more comments to clarify the implicit convention of state modification.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373916473
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
+                throw new RuntimeException("Already closed");
+            }
 
 Review comment:
   I prefer to RuntimeException. In my opinion, IOException indicates the actual physical network is broken. But I think it is not proper to throw IOException here. Here it is software issue that misses the state check. The direct reason is not a real physical network broken.  I think that throwing of IOException here would confuse the users. So I still prefer to use RuntimeException here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] eolivelli commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r385525037
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,333 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.ClientCnxn.Packet;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ClientCnxnSocketFragilityTest extends QuorumPeerTestBase {
+
+    private static final int SERVER_COUNT = 3;
+
+    private static final int SESSION_TIMEOUT = 40000;
+
+    public static final int CONNECTION_TIMEOUT = 30000;
+
+    private final UnsafeCoordinator unsafeCoordinator = new UnsafeCoordinator();
+
+    private volatile CustomZooKeeper zk = null;
+
+    private volatile FragileClientCnxnSocketNIO socket = null;
+
+    private volatile CustomClientCnxn cnxn = null;
+
+    private String getCxnString(int[] clientPorts) {
+        StringBuffer hostPortBuffer = new StringBuffer();
+        for (int i = 0; i < clientPorts.length; i++) {
+            hostPortBuffer.append("127.0.0.1:");
+            hostPortBuffer.append(clientPorts[i]);
+            if (i != (clientPorts.length - 1)) {
+                hostPortBuffer.append(',');
+            }
+        }
+        return hostPortBuffer.toString();
+    }
+
+    private void closeZookeeper(ZooKeeper zk) {
+        Executors.newSingleThreadExecutor().submit(() -> {
+            try {
+                LOG.info("closeZookeeper is fired");
+                zk.close();
+            } catch (InterruptedException e) {
+            }
+        });
+    }
+
+    private void getDataBackgroundRetryForever(CustomZooKeeper zk, String path) {
+        new Thread(() -> {
+            for (;;) {
+                try {
+                    zk.getData(path, false, new Stat());
+                } catch (Exception e) {
 
 Review comment:
   Please log something, without the stacktrace

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373362709
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -782,6 +782,10 @@ void queueCallback(AsyncCallback cb, int rc, String path, Object ctx) {
         eventThread.queueCallback(cb, rc, path, ctx);
     }
 
+    protected void onConnecting(InetSocketAddress addr) {
+        LOG.info("Connecting server {}.", addr);
+    }
+
 
 Review comment:
   - I wonder whether a log is needed here? because I found the `logStartConnect` in the `SendThread#startConnect`?
   - If needed, `LOG.info("Start to reconnect to server {}.", addr); `or `LOG.info("Try to reconnect to server {}.", addr)`; is better?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r374471742
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
 
 Review comment:
   synchronized(state) is still not safe. Maybe SendThread.close() and SendThread.startConnect() synchronize on different objects because state is mutable and can be changed between SendThread.close() and SendThread.startConnect().
   I prefer synchronized(this) guard.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r370933641
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,310 @@
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 
 Review comment:
   Some imports are never used, also notice the useless field in this class(e.g: `scheduledExecutor`)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r386180905
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1428,7 +1447,10 @@ void onConnected(
         }
 
         void close() {
-            state = States.CLOSED;
+            try {
+                changeZkState(States.CLOSED);
+            } catch (IOException e) {
 
 Review comment:
   Thanks for your suggestion. Accept your suggestion. I add one log here. Please review the latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r381248377
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   Thank you for your comments.
   You convince me. Accept it. Add `synchronized` back to getZkState()

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379970390
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   Thank you for your comments.
   In my memory, checkstyle may report exception if remove `synchronized` from getkState. But I can not reproduce the checkstyle error in current version. Maybe I missed something. 
   Another reason of adding `synchronized` is to prevent future risk.
   In current implementation, we have implicit convention for setZkSate() - state modification must be placed at the end of setZkState() otherwise incomplete internal state may be exposed to outside. Adding `synchronized` to getZkState() can avoid this risk because getZkState() only be allowed to interleave execution with the whole setZkState().
   
   Anyway I accept your suggestion. Maybe it has potential performance benefit if I remove `synchronized` from getZkState(). I will add more comments to clarify the implicit convention of state modification.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] asfgit closed pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-581224808
 
 
   > @yfxhust
   > Great issue description in JIRA. this issue is a little hidden. Do you observe it in the production?
   
   We observed occasional SendThread leak case in our production. But the root cause is 100 percent sure. But from source code and some network clues, we believe that this issue is a potential cause.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-588192138
 
 
   > I just have two nits - one on the sync approach and the other on unit test.
   
   Thank you for your comments. Adapt my code with your suggestion. Please see the latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] maoling commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
maoling commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-580634076
 
 
   @yfxhust
   Great issue description in JIRA. this issue is a little hidden. Do you observe it in the production?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r381084467
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.ClientCnxn.Packet;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ClientCnxnSocketFragilityTest extends QuorumPeerTestBase {
+
+    private static final int SERVER_COUNT = 3;
+
+    private static final int SESSION_TIMEOUT = 40000;
+
+    public static final int CONNECTION_TIMEOUT = 30000;
+
+    private final UnsafeCoordinator unsafeCoordinator = new UnsafeCoordinator();
+
+    private volatile CustomZooKeeper zk = null;
+
+    private volatile FragileClientCnxnSocketNIO socket = null;
+
+    private volatile CustomClientCnxn cnxn = null;
+
+    private String getCxnString(int[] clientPorts) {
+        StringBuffer hostPortBuffer = new StringBuffer();
+        for (int i = 0; i < clientPorts.length; i++) {
+            hostPortBuffer.append("127.0.0.1:");
+            hostPortBuffer.append(clientPorts[i]);
+            if (i != (clientPorts.length - 1)) {
+                hostPortBuffer.append(',');
+            }
+        }
+        return hostPortBuffer.toString();
+    }
+
+    private void closeZookeeper(ZooKeeper zk) {
+        Executors.newSingleThreadExecutor().submit(() -> {
+            try {
+                LOG.info("closeZookeeper is fired");
+                zk.close();
+            } catch (InterruptedException e) {
+            }
+        });
+    }
+
+    private void getDataBackgroundRetryForever(CustomZooKeeper zk, String path) {
+        new Thread(() -> {
+            for (;;) {
+                try {
+                    zk.getData(path, false, new Stat());
+                } catch (Exception e) {
+                }
+            }
+        }).start();
+    }
+
+    @Test
+    public void testClientCnxnSocketFragility() throws Exception {
+        System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, FragileClientCnxnSocketNIO.class.getName());
+        System.setProperty(ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT, "1000");
+        final int[] clientPorts = new int[SERVER_COUNT];
+        StringBuilder sb = new StringBuilder();
+        String server;
+
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            clientPorts[i] = PortAssignment.unique();
+            server = "server." + i + "=127.0.0.1:" + PortAssignment.unique() + ":" + PortAssignment.unique()
+                     + ":participant;127.0.0.1:" + clientPorts[i];
+            sb.append(server + "\n");
+        }
+        String currentQuorumCfgSection = sb.toString();
+        MainThread[] mt = new MainThread[SERVER_COUNT];
+
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, false);
+            mt[i].start();
+        }
+
+        // Ensure server started
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            Assert.assertTrue(
+                "waiting for server " + i + " being up",
+                ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], CONNECTION_TIMEOUT));
+        }
+        String path = "/testClientCnxnSocketFragility";
+        String data = "balabala";
+        ClientWatcher watcher = new ClientWatcher();
+        zk = new CustomZooKeeper(getCxnString(clientPorts), SESSION_TIMEOUT, watcher);
+        watcher.watchFor(zk);
+
+        // Let's see some successful operations
+        zk.create(path, data.getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        Assert.assertEquals(new String(zk.getData(path, false, new Stat())), data);
+        Assert.assertTrue(!watcher.isSessionExpired());
+
+        // Let's make a broken operation
+        socket.mute();
+        boolean catchKeeperException = false;
+        try {
+            zk.getData(path, false, new Stat());
+        } catch (KeeperException e) {
+            catchKeeperException = true;
+            Assert.assertFalse(e instanceof KeeperException.SessionExpiredException);
+        }
+        socket.unmute();
+        Assert.assertTrue(catchKeeperException);
+        Assert.assertTrue(!watcher.isSessionExpired());
+
+        getDataBackgroundRetryForever(zk, path);
+        // Let's make a broken network
+        socket.mute();
+
+        // Let's attempt to close ZooKeeper
+        cnxn.attemptClose();
+
+        // Wait some time to expect continuous reconnecting.
+        // We try to make reconnecting hit the unsafe point.
+        TimeUnit.MILLISECONDS.sleep(3000);
 
 Review comment:
   Is it possible to instead of using sleep, we do explicit synchronization and only execute when certain criteria is satisfied? We generally discourage the usage of sleep in zookeeper unit test as the use of sleep proved to be a source of flaky-ness in the past when running on different environment.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379962719
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
         }
 
+        synchronized void setZkState(ZooKeeper.States newState) throws IOException {
+            if (!state.isAlive()) {
+                if (newState == States.CONNECTING) {
+                    throw new IOException(
 
 Review comment:
   Current implementation seems more structural for future evaluation because it will handle more scenarios.
   But anyway I accept your suggestion. Currently your suggestion seems more compact.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379963881
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1179,6 +1198,7 @@ public void run() {
                         } else {
                             serverAddress = hostProvider.next(1000);
                         }
+                        ClientCnxn.this.onConnecting(serverAddress);
                         startConnect(serverAddress);
 
 Review comment:
   Accept it. Good suggestion.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373916473
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
+                throw new RuntimeException("Already closed");
+            }
 
 Review comment:
   I prefer RuntimeException. In my opinion, IOException indicates the actual physical network is broken. But I think it is not proper to throw IOException here. Here it is software issue that misses the state check. The direct reason is not a real physical network broken.  I think that throwing of IOException here would confuse the users. So I still prefer to use RuntimeException here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-588043620
 
 
   I just have two nits - one on the sync approach and the other on unit test. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379822792
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1179,6 +1198,7 @@ public void run() {
                         } else {
                             serverAddress = hostProvider.next(1000);
                         }
+                        ClientCnxn.this.onConnecting(serverAddress);
                         startConnect(serverAddress);
 
 Review comment:
   `onConnecting(serverAddress);`  ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r381250718
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.ClientCnxn.Packet;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ClientCnxnSocketFragilityTest extends QuorumPeerTestBase {
+
+    private static final int SERVER_COUNT = 3;
+
+    private static final int SESSION_TIMEOUT = 40000;
+
+    public static final int CONNECTION_TIMEOUT = 30000;
+
+    private final UnsafeCoordinator unsafeCoordinator = new UnsafeCoordinator();
+
+    private volatile CustomZooKeeper zk = null;
+
+    private volatile FragileClientCnxnSocketNIO socket = null;
+
+    private volatile CustomClientCnxn cnxn = null;
+
+    private String getCxnString(int[] clientPorts) {
+        StringBuffer hostPortBuffer = new StringBuffer();
+        for (int i = 0; i < clientPorts.length; i++) {
+            hostPortBuffer.append("127.0.0.1:");
+            hostPortBuffer.append(clientPorts[i]);
+            if (i != (clientPorts.length - 1)) {
+                hostPortBuffer.append(',');
+            }
+        }
+        return hostPortBuffer.toString();
+    }
+
+    private void closeZookeeper(ZooKeeper zk) {
+        Executors.newSingleThreadExecutor().submit(() -> {
+            try {
+                LOG.info("closeZookeeper is fired");
+                zk.close();
+            } catch (InterruptedException e) {
+            }
+        });
+    }
+
+    private void getDataBackgroundRetryForever(CustomZooKeeper zk, String path) {
+        new Thread(() -> {
+            for (;;) {
+                try {
+                    zk.getData(path, false, new Stat());
+                } catch (Exception e) {
+                }
+            }
+        }).start();
+    }
+
+    @Test
+    public void testClientCnxnSocketFragility() throws Exception {
+        System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, FragileClientCnxnSocketNIO.class.getName());
+        System.setProperty(ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT, "1000");
+        final int[] clientPorts = new int[SERVER_COUNT];
+        StringBuilder sb = new StringBuilder();
+        String server;
+
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            clientPorts[i] = PortAssignment.unique();
+            server = "server." + i + "=127.0.0.1:" + PortAssignment.unique() + ":" + PortAssignment.unique()
+                     + ":participant;127.0.0.1:" + clientPorts[i];
+            sb.append(server + "\n");
+        }
+        String currentQuorumCfgSection = sb.toString();
+        MainThread[] mt = new MainThread[SERVER_COUNT];
+
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, false);
+            mt[i].start();
+        }
+
+        // Ensure server started
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            Assert.assertTrue(
+                "waiting for server " + i + " being up",
+                ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], CONNECTION_TIMEOUT));
+        }
+        String path = "/testClientCnxnSocketFragility";
+        String data = "balabala";
+        ClientWatcher watcher = new ClientWatcher();
+        zk = new CustomZooKeeper(getCxnString(clientPorts), SESSION_TIMEOUT, watcher);
+        watcher.watchFor(zk);
+
+        // Let's see some successful operations
+        zk.create(path, data.getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        Assert.assertEquals(new String(zk.getData(path, false, new Stat())), data);
+        Assert.assertTrue(!watcher.isSessionExpired());
+
+        // Let's make a broken operation
+        socket.mute();
+        boolean catchKeeperException = false;
+        try {
+            zk.getData(path, false, new Stat());
+        } catch (KeeperException e) {
+            catchKeeperException = true;
+            Assert.assertFalse(e instanceof KeeperException.SessionExpiredException);
+        }
+        socket.unmute();
+        Assert.assertTrue(catchKeeperException);
+        Assert.assertTrue(!watcher.isSessionExpired());
+
+        getDataBackgroundRetryForever(zk, path);
+        // Let's make a broken network
+        socket.mute();
+
+        // Let's attempt to close ZooKeeper
+        cnxn.attemptClose();
+
+        // Wait some time to expect continuous reconnecting.
+        // We try to make reconnecting hit the unsafe point.
+        TimeUnit.MILLISECONDS.sleep(3000);
 
 Review comment:
   Thanks for your comments.
   Agree with you. I add CustomClientCnxn.waitUntilHitUnsafeRegion(). It ensures the unsafe region is hit. Please see the latest patch. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r386194266
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,333 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.ClientCnxn.Packet;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ClientCnxnSocketFragilityTest extends QuorumPeerTestBase {
+
+    private static final int SERVER_COUNT = 3;
+
+    private static final int SESSION_TIMEOUT = 40000;
+
+    public static final int CONNECTION_TIMEOUT = 30000;
+
+    private final UnsafeCoordinator unsafeCoordinator = new UnsafeCoordinator();
+
+    private volatile CustomZooKeeper zk = null;
+
+    private volatile FragileClientCnxnSocketNIO socket = null;
+
+    private volatile CustomClientCnxn cnxn = null;
+
+    private String getCxnString(int[] clientPorts) {
+        StringBuffer hostPortBuffer = new StringBuffer();
+        for (int i = 0; i < clientPorts.length; i++) {
+            hostPortBuffer.append("127.0.0.1:");
+            hostPortBuffer.append(clientPorts[i]);
+            if (i != (clientPorts.length - 1)) {
+                hostPortBuffer.append(',');
+            }
+        }
+        return hostPortBuffer.toString();
+    }
+
+    private void closeZookeeper(ZooKeeper zk) {
+        Executors.newSingleThreadExecutor().submit(() -> {
+            try {
+                LOG.info("closeZookeeper is fired");
+                zk.close();
+            } catch (InterruptedException e) {
+            }
+        });
+    }
+
+    private void getDataBackgroundRetryForever(CustomZooKeeper zk, String path) {
+        new Thread(() -> {
+            for (;;) {
 
 Review comment:
   Thanks for your comments. Accept your suggestion. Now I delete this task to a new class - GetDataRetryForeverBackgroundTask. I mark GetDataRetryForeverBackgroundTask as deamon thread and handle graceful shutdown in syncCloseTask(). I think these adoption can avoid the CPU exhaust. Please review the latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373914530
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -782,6 +782,10 @@ void queueCallback(AsyncCallback cb, int rc, String path, Object ctx) {
         eventThread.queueCallback(cb, rc, path, ctx);
     }
 
+    protected void onConnecting(InetSocketAddress addr) {
+        LOG.info("Connecting server {}.", addr);
+    }
+
 
 Review comment:
   Actually onConnecting() method is added to facilitate the test case. onConnecting() is placed at the unsafe point that is described in JIRA issue. We can override this method to inject some actions in test case. The injected actions make issue happen. If we don't have this method, it is difficult to construct a test case to reproduce the issue.
   Your suggestion of log changing is good for me. Accept it. I will change the log to "Try to reconnect to server {}." Thanks for your comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r374466001
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
 
 Review comment:
   Thanks for your comments. I agree with your concern. I would add synchronized(state) to ensure the atomic operation. I also add synchronized(state) guard at SendThread.close() to set state as States.CLOSED. I think these guards can resolve the problem.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-582243797
 
 
   left two more comments - current approach should work but I am wondering if my proposal would sound better in term of more structurally solve the race condition around the state variable; also was a little bit concerning around RuntimeException we throw and the implications it brought (w.r.t resource leaks)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r375734071
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
+                throw new RuntimeException("Already closed");
+            }
 
 Review comment:
   Both of you convince me. Now I throw IOException in SendThread.setZkState. Please check the latest patch 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r375732546
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
 
 Review comment:
   Accept it. Now I make a new method SendThread.setZkState() with checked IOException. Please check the latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r370933702
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,310 @@
+package org.apache.zookeeper;
 
 Review comment:
   LICENSE ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-582822785
 
 
   > left two more comments - current approach should work but I am wondering if my proposal would sound better in term of more structurally solve the race condition around the state variable; also was a little bit concerning around RuntimeException we throw and the implications it brought (w.r.t resource leaks)
   
   Thank you for your comments. I adapt my code with your suggestions. Please review the latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r386194266
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,333 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.ClientCnxn.Packet;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ClientCnxnSocketFragilityTest extends QuorumPeerTestBase {
+
+    private static final int SERVER_COUNT = 3;
+
+    private static final int SESSION_TIMEOUT = 40000;
+
+    public static final int CONNECTION_TIMEOUT = 30000;
+
+    private final UnsafeCoordinator unsafeCoordinator = new UnsafeCoordinator();
+
+    private volatile CustomZooKeeper zk = null;
+
+    private volatile FragileClientCnxnSocketNIO socket = null;
+
+    private volatile CustomClientCnxn cnxn = null;
+
+    private String getCxnString(int[] clientPorts) {
+        StringBuffer hostPortBuffer = new StringBuffer();
+        for (int i = 0; i < clientPorts.length; i++) {
+            hostPortBuffer.append("127.0.0.1:");
+            hostPortBuffer.append(clientPorts[i]);
+            if (i != (clientPorts.length - 1)) {
+                hostPortBuffer.append(',');
+            }
+        }
+        return hostPortBuffer.toString();
+    }
+
+    private void closeZookeeper(ZooKeeper zk) {
+        Executors.newSingleThreadExecutor().submit(() -> {
+            try {
+                LOG.info("closeZookeeper is fired");
+                zk.close();
+            } catch (InterruptedException e) {
+            }
+        });
+    }
+
+    private void getDataBackgroundRetryForever(CustomZooKeeper zk, String path) {
+        new Thread(() -> {
+            for (;;) {
 
 Review comment:
   Thanks for your comments. Accept your suggestion. Now I delegate this task to a new class - GetDataRetryForeverBackgroundTask. I mark GetDataRetryForeverBackgroundTask as deamon thread and handle graceful shutdown in syncCloseTask(). I think these adoption can avoid the CPU exhaust. Please review the latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373906397
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,310 @@
+package org.apache.zookeeper;
 
 Review comment:
   Thanks for comment. Accept it. I will add LICENSE statement in the test case.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r374466290
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -782,6 +782,10 @@ void queueCallback(AsyncCallback cb, int rc, String path, Object ctx) {
         eventThread.queueCallback(cb, rc, path, ctx);
     }
 
+    protected void onConnecting(InetSocketAddress addr) {
+        LOG.info("Try to connect to server {}.", addr);
 
 Review comment:
   Thanks your comments. Accept it. I will add comments to declare it as "test only" and leave this method as empty.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373914530
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -782,6 +782,10 @@ void queueCallback(AsyncCallback cb, int rc, String path, Object ctx) {
         eventThread.queueCallback(cb, rc, path, ctx);
     }
 
+    protected void onConnecting(InetSocketAddress addr) {
+        LOG.info("Connecting server {}.", addr);
+    }
+
 
 Review comment:
   Actually onConnecting() method is added to facilitate the test case. onConnecting() is placed at the unsafe point that is described in JIRA issue. We can override this method to inject some actions in test case. The injected actions make issue happen. If we don't have this method, it is difficult to construct a test case to reproduce the issue.
   Your suggestion of log changing is good for me. Accept it. I will change the log to "Try to connect to server {}."  I change the word "reconnect" to "connect" in your word. I think "connect" is better because the first connect also can hit here. The first connect attempt should not be treated as "reconnect".  
   Thanks for your comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-581734622
 
 
   > nice work - left two comments. since the request timeout is a feature disabled by default, the impact of this bug is limited to those who enabled this feature only.
   
   Thanks for your comments

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-581653852
 
 
   nice work - left two comments. since the request timeout is a feature disabled by default, the impact of this bug is limited to those who enabled this feature only. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379970390
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   Thank you for your comments.
   checkstyle may report exception if remove `synchronized` from getkState. 
   Another reason of adding `synchronized` is to prevent future risk if we add more complex task in setZkState() in future..
   In current implementation, we have implicit convention for setZkSate() - state modification must be placed at the end of setZkState() otherwise incomplete internal state may be exposed to outside. Adding `synchronized` to getZkState() can avoid this risk because getZkState() only be allowed to interleave execution with the whole setZkState().
   
   Anyway I accept your suggestion. Maybe it has potential performance benefit if I remove `synchronized` from getZkState(). I will add more comments to clarify the implicit convention of state modification.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r386194563
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,333 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.ClientCnxn.Packet;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ClientCnxnSocketFragilityTest extends QuorumPeerTestBase {
+
+    private static final int SERVER_COUNT = 3;
+
+    private static final int SESSION_TIMEOUT = 40000;
+
+    public static final int CONNECTION_TIMEOUT = 30000;
+
+    private final UnsafeCoordinator unsafeCoordinator = new UnsafeCoordinator();
+
+    private volatile CustomZooKeeper zk = null;
+
+    private volatile FragileClientCnxnSocketNIO socket = null;
+
+    private volatile CustomClientCnxn cnxn = null;
+
+    private String getCxnString(int[] clientPorts) {
+        StringBuffer hostPortBuffer = new StringBuffer();
+        for (int i = 0; i < clientPorts.length; i++) {
+            hostPortBuffer.append("127.0.0.1:");
+            hostPortBuffer.append(clientPorts[i]);
+            if (i != (clientPorts.length - 1)) {
+                hostPortBuffer.append(',');
+            }
+        }
+        return hostPortBuffer.toString();
+    }
+
+    private void closeZookeeper(ZooKeeper zk) {
+        Executors.newSingleThreadExecutor().submit(() -> {
+            try {
+                LOG.info("closeZookeeper is fired");
+                zk.close();
+            } catch (InterruptedException e) {
+            }
+        });
+    }
+
+    private void getDataBackgroundRetryForever(CustomZooKeeper zk, String path) {
+        new Thread(() -> {
+            for (;;) {
+                try {
+                    zk.getData(path, false, new Stat());
+                } catch (Exception e) {
 
 Review comment:
   Accept it. Add one info log: zookeeper getData failed on path {}. Please review the latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-593730437
 
 
   merged to master - thanks for driving this @yfxhust 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r374379920
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
 
 Review comment:
   I suspect this will not completely fix the race condition between setting the state to closed and connecting. Another thread could set the state to close immediately after this check but before the subsequent `state = States.CONNECTING;` statement. We could make sure that this "check and assign" operation is atomic to completely address the race condition.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379962719
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
         }
 
+        synchronized void setZkState(ZooKeeper.States newState) throws IOException {
+            if (!state.isAlive()) {
+                if (newState == States.CONNECTING) {
+                    throw new IOException(
 
 Review comment:
   Thank you for your comments.
   Current implementation seems more structural for future evaluation because it will handle more scenarios.
   But anyway I accept your suggestion. Your suggestion seems more compact.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r375060653
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
 
 Review comment:
   I think this will work. 
   
   I would recommend instead of doing the check here, we add a set of new `synchronized` methods to `SenderThread` where we get and set the states of the `SenderThread.state`. Then, we can move the checks in the `setState` method, where we can put constraints on how the connection state may transit. For example, we can enforce in that method that a `CLOSED` state can't be transit to a different state, which will prevent the problem we are seeing in this issue (that setting a state to `CONNECTING` after being in `CLOSED` state). This has additional benefits as currently, there are multiple write paths to the `SenderThread.state`, and by consolidate read and write access of the state in these synchronized methods, we can be sure there will not be future race conditions w.r.t accessing this state variable.
   
   I also thought about an alternative - instead of synchronizing on `SenderThread`, we could wrap the state variable in an `AtomicReference` and then provide get/set/compareAndSet methods on top of the atomic reference. This however is less flexible to allow us to control how the state could transit, so it still looks to me that  synchronizing on `SenderThread` is better.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] eolivelli commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r385524422
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1428,7 +1447,10 @@ void onConnected(
         }
 
         void close() {
-            state = States.CLOSED;
+            try {
+                changeZkState(States.CLOSED);
+            } catch (IOException e) {
 
 Review comment:
   Please add a comment
   We could anyway spend a line of log, thus case won't happen but if it is happening we won't know

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379822895
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   `state` is already modified by `volatile`, so `synchronized` is no need ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
maoling commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373364767
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
+                throw new RuntimeException("Already closed");
+            }
 
 Review comment:
   Just as you said, this issue happens when network is broken, so `IOException` is better? e.g. 
   `throw new IOException("Socket has already closed when reconnecting to "+ addr);`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r375061974
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
+                throw new RuntimeException("Already closed");
+            }
 
 Review comment:
   `RuntimeException` indicates unrecoverable programming error - such as illegal arguments or unrecoverable errors. Our case here is not that dramatic, it's recoverable by throwing an IOException if the state is `CLOSED` and it's being set to another value. `CLOSED` state will signal `SenderThread` to terminate and the benefit is we will be able to clean up resources - like the sockets sender thread held, and more importantly, the event thread will also be properly shutdown. 
   And since we are explicitly programming in this issue to prevent this error, it does not sound like a logic error to me that worth to throw a RuntimeException.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379962719
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
         }
 
+        synchronized void setZkState(ZooKeeper.States newState) throws IOException {
+            if (!state.isAlive()) {
+                if (newState == States.CONNECTING) {
+                    throw new IOException(
 
 Review comment:
   Thank you for your comments.
   Current implementation seems more structural for future evaluation because it will handle more scenarios.
   But anyway I accept your suggestion. Currently your suggestion seems more compact.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379970390
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   Thank you for your comments.
   spotbugs-maven-plugin:3.1.9:check may report error for UG_SYNC_SET_UNSYNC_GET if remove `synchronized` from getZkState because we have synchronized setZkState. 
   Another reason of adding `synchronized` is to prevent future risk if we add more complex task in setZkState() in future.
   In current implementation, we have implicit convention for setZkSate() - state modification must be placed at the end of setZkState() otherwise incomplete internal state may be exposed to outside. Adding `synchronized` to getZkState() can avoid this risk because getZkState() only be allowed to interleave execution with the whole setZkState().
   
   Anyway I accept your suggestion. Maybe it has potential performance benefit if I remove `synchronized` from getZkState(). I will add more comments to clarify the implicit convention of state modification and rename setZkState() to changZkState to make spotbugs-maven-plugin happy.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r375734071
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
+                throw new RuntimeException("Already closed");
+            }
 
 Review comment:
   Both of you convince me. Now I throw IOException in SendThread.setZkState. Please check latest patch 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r373900060
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java
 ##########
 @@ -0,0 +1,310 @@
+package org.apache.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Queue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 
 Review comment:
   Thanks for comment. Accept it. I miss the check for unused field. I will fix it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r379970390
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   Thank you for your comments.
   spotbugs-maven-plugin:3.1.9:check may report error if remove `synchronized` from getkState. 
   Another reason of adding `synchronized` is to prevent future risk if we add more complex task in setZkState() in future..
   In current implementation, we have implicit convention for setZkSate() - state modification must be placed at the end of setZkState() otherwise incomplete internal state may be exposed to outside. Adding `synchronized` to getZkState() can avoid this risk because getZkState() only be allowed to interleave execution with the whole setZkState().
   
   Anyway I accept your suggestion. Maybe it has potential performance benefit if I remove `synchronized` from getZkState(). I will add more comments to clarify the implicit convention of state modification and rename setZkState

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r375734071
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -1121,6 +1125,9 @@ private void startConnect(InetSocketAddress addr) throws IOException {
                     LOG.warn("Unexpected exception", e);
                 }
             }
+            if (!state.isAlive()) {
+                throw new RuntimeException("Already closed");
+            }
 
 Review comment:
   Both of you convince me. Now I throw IOException in SendThread.setZkState(). Please check the latest patch 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#discussion_r381083959
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
 ##########
 @@ -971,10 +980,20 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
          *
          * @return
          */
-        ZooKeeper.States getZkState() {
+        synchronized ZooKeeper.States getZkState() {
             return state;
 
 Review comment:
   I actually prefer the previous synchronized getZkState() approach - it's simple and consistent and easy to understand than the current approach. It's good to have performance in mind, but unless there is a real bottleneck, I would always choose simple code over premature optimization.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…

Posted by GitBox <gi...@apache.org>.
yfxhust commented on issue #1235: ZOOKEEPER-3706: ZooKeeper.close() would leak SendThread when the netw…
URL: https://github.com/apache/zookeeper/pull/1235#issuecomment-593218845
 
 
   @eolivelli Thank you for your comments. I adapt the patch with your suggestions. Please review the latest patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services