You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by yufeldman <gi...@git.apache.org> on 2016/11/04 22:22:44 UTC

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

GitHub user yufeldman opened a pull request:

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

    ZOOKEEPER-2549 Add exception handling to sendResponse

    Fix for:
    As NettyServerCnxn.sendResponse() allows all the exception to bubble up it can stop main ZK requests processing thread
    Same changes done for NIOServerCnxn

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

    $ git pull https://github.com/yufeldman/zookeeper master2549

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

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

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

    This closes #99
    
----
commit 18270e28f3e60a03422641dd0ee2ccaad6a3b096
Author: Yuliya Feldman <yf...@maprtech.com>
Date:   2016-09-03T05:18:30Z

    ZOOKEEPER-2549 Add exception handling to sendResponse

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92326078
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIONoException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  private void testZKNoExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "NoException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(path, false);
    +      if ( stats != null ) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.error("ConnectionLossException: {}", cle);
    +      fail("No exception should be thrown");
    +    } catch (Throwable t) {
    +      // error
    +      LOG.error("Throwable {}", t);
    +      fail("No exception should be thrown");
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +  @Test (timeout = 10000)
    --- End diff --
    
    see above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #99: ZOOKEEPER-2549 Add exception handling to sendResponse

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on the issue:

    https://github.com/apache/zookeeper/pull/99
  
    Addressed review comments


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #99: ZOOKEEPER-2549 Add exception handling to sendResponse

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on the issue:

    https://github.com/apache/zookeeper/pull/99
  
    @fpj, @hanm could you take a look? I am +1, but another look would be nice.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90963562
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    Interesting. For me it actually is:
    
    2016-12-05 11:11:02,523 [myid:] - INFO  [Time-limited test:JUnit4ZKTestRunner$LoggedInvokeMethod@98] - TEST METHOD FAILED testNettyRunTimeException
    **java.io.IOException: Couldn't instantiate org.apache.zookeeper.server.NettyServerCnxnFactory
            at org.apache.zookeeper.server.ServerCnxnFactory.createFactory(ServerCnxnFactory.java:141)
            at org.apache.zookeeper.server.ServerCnxnFactory.createFactory(ServerCnxnFactory.java:157)
            at org.apache.zookeeper.server.ServerCnxnFactory.createFactory(ServerCnxnFactory.java:151)**



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92296453
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    --- End diff --
    
    Aren't we missing a ``try-catch`` around ``zk.close()`` with an explicit comment to ignore any error message?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92298117
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -694,7 +690,7 @@ public void sendResponse(ReplyHeader h, Record r, String tag) {
                     }
                 }
              } catch(Exception e) {
    -            LOG.warn("Unexpected exception. Destruction averted.", e);
    +            throw new IOException(e);
    --- End diff --
    
    Just a suggestion: wdyt about bubbling up a more custom message with the IOException instead of just encapsulate the Exception? I mean, something like:
    
    ``
    throw new IOException("sendMessage exception: blah blah", e);
    ``


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90953900
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    @hanm - >>> so all tests fall back to create NIOServerCnxnx instead
    It is not the case, they don't fall back to NIO - they fail.
    Here is printout form running tests with both Netty and MockNettyServerCnxn not having public ctor:
       [junit] Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 0.159 sec
      **  [junit] Test org.apache.zookeeper.server.NettyServerCnxnTest FAILED **
        [junit] Running org.apache.zookeeper.server.PrepRequestProcessorTest
        [junit] Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.279 sec
        [junit] Running org.apache.zookeeper.server.PurgeTxnTest
        [junit] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 2.431 sec
        [junit] Running org.apache.zookeeper.server.ReferenceCountedACLCacheTest
        [junit] Tests run: 10, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.089 sec
        [junit] Running org.apache.zookeeper.server.SerializationPerfTest
        [junit] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 1.114 sec
        [junit] Running org.apache.zookeeper.server.ServerCxnExceptionsTest
        [junit] Tests run: 6, Failures: 0, Errors: 6, Skipped: 0, Time elapsed: 0.284 sec
    **    [junit] Test org.apache.zookeeper.server.ServerCxnExceptionsTest FAILED **
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #99: ZOOKEEPER-2549 Add exception handling to sendResponse

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on the issue:

    https://github.com/apache/zookeeper/pull/99
  
    I like this patch, but I think the whole reflection/mock thing is kind of reinventing a fault injection inside the test classes. If so, why not use a production ready framework as Byteman? I wrote a PR that strips the boilerplate stuff while leaving the feature of this PR: https://github.com/apache/zookeeper/pull/123 
    
    Still a PoC, so any suggestions are welcome. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92268597
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    --- End diff --
    
    I didn't get why we need the lines 87-90, because ``fail()`` throws an ``AssertionError`` that will interrupt the processing flow, so those lines are effectively unreachable, right? There's should be nothing more after the `fail()`, I **guess**.
    
    Also, I would suggest to put a more meaningful message, something along the lines of ``sendResponse should have thrown IOException and failed this test.``


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90935413
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    @hanm - there is nothing wrong with the tests, they do run fine. They do not use ctor from NettyServerCnxn, but from MockNettyServerCnxn - that has public ctor.
    You can make MockNettyServerCnxn not public and you will have the same issue. And BTW 
    org.apache.zookeeper.server.NettyServerCnxnTest fails with not public ctor in NettyServerCnxn 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90782724
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    +            close();
    --- End diff --
    
    Will remove close() from catch block


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86679205
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java ---
    @@ -842,7 +868,26 @@ private void addCnxn(NIOServerCnxn cnxn) {
     
         protected NIOServerCnxn createConnection(SocketChannel sock,
                 SelectionKey sk, SelectorThread selectorThread) throws IOException {
    -        return new NIOServerCnxn(zkServer, sock, sk, this, selectorThread);
    +
    +        NIOServerCnxn cnxn = null;
    +
    +        if (serverCnxnClassCtr != null) {
    --- End diff --
    
    what's wrong with setting serverCnxnClassCtr to NettyServerCnxn by default (by doing all the reflection magic)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86672337
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    --- End diff --
    
    i have mixed feelings with concatenating strings in a hot path (IOException happening here is a hot path when, for instance, a network blip happens). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90963807
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    @hanm - thank you for all the reviews and feedback. Really appreciate it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92266304
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    --- End diff --
    
    typo: ``scenarios``


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86672349
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java ---
    @@ -630,6 +640,22 @@ public static ByteBuffer getDirectBuffer() {
          * limits of the operating system). startup(zks) must be called subsequently.
          */
         public NIOServerCnxnFactory() {
    +        String serverCnxnClassName = System.getProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +        if ( serverCnxnClassName != null ) {
    --- End diff --
    
    nit: extra spaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92264074
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    --- End diff --
    
    nit: I would use a modern debug format:
    
    ``
    LOG.debug("Problem sending to {}", getRemoteSocketAddress(), ex);
    ``


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90762620
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    It is not obvious to me why the access specifier of `NettyServerCnxn` should be changed public here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86672326
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    +            close();
    --- End diff --
    
    this is an interesting change -- how/when were we closing the connection before?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86674430
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    --- End diff --
    
    Any suggestion here? Not to getRemoteSocketAddress() at all?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86691054
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java ---
    @@ -842,7 +868,26 @@ private void addCnxn(NIOServerCnxn cnxn) {
     
         protected NIOServerCnxn createConnection(SocketChannel sock,
                 SelectionKey sk, SelectorThread selectorThread) throws IOException {
    -        return new NIOServerCnxn(zkServer, sock, sk, this, selectorThread);
    +
    +        NIOServerCnxn cnxn = null;
    +
    +        if (serverCnxnClassCtr != null) {
    --- End diff --
    
    nothing is wrong with it per say - just overhead, as we will call "newInstance" on the constructor for every code path - test or not.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92297692
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIONoException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  private void testZKNoExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "NoException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(path, false);
    +      if ( stats != null ) {
    --- End diff --
    
    I don't get what we are even bothering of adding lines 122-124. Why?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92269582
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    --- End diff --
    
    Method name doesn't follow camel case convention (``nettySetup`` instead of ``NettySetup``).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86679164
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    +            close();
    --- End diff --
    
    hmm, that would be a pretty bad leak given all what goes on in close():
    
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90550930
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -165,31 +163,35 @@ public void process(WatchedEvent event) {
         @Override
         public void sendResponse(ReplyHeader h, Record r, String tag)
                 throws IOException {
    -        if (!channel.isOpen()) {
    -            return;
    -        }
    -        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    -        // Make space for length
    -        BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
             try {
    -            baos.write(fourBytes);
    -            bos.writeRecord(h, "header");
    -            if (r != null) {
    -                bos.writeRecord(r, tag);
    +            if (!channel.isOpen()) {
    +                return;
                 }
    -            baos.close();
    -        } catch (IOException e) {
    -            LOG.error("Error serializing response");
    -        }
    -        byte b[] = baos.toByteArray();
    -        ByteBuffer bb = ByteBuffer.wrap(b);
    -        bb.putInt(b.length - 4).rewind();
    -        sendBuffer(bb);
    -        if (h.getXid() > 0) {
    -            // zks cannot be null otherwise we would not have gotten here!
    -            if (!zkServer.shouldThrottle(outstandingCount.decrementAndGet())) {
    -                enableRecv();
    +            ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +            // Make space for length
    +            BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
    +            try {
    +                baos.write(fourBytes);
    +                bos.writeRecord(h, "header");
    +                if (r != null) {
    +                    bos.writeRecord(r, tag);
    +                }
    +                baos.close();
    +            } catch (IOException e) {
    --- End diff --
    
    This IOException is swallowed either, should we re-throw it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86672419
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java ---
    @@ -105,8 +115,26 @@ public void channelConnected(ChannelHandlerContext ctx,
                     LOG.trace("Channel connected " + e);
                 }
     
    -            NettyServerCnxn cnxn = new NettyServerCnxn(ctx.getChannel(),
    -                    zkServer, NettyServerCnxnFactory.this);
    +            NettyServerCnxn cnxn = null;
    +            if (serverCnxnClassCtr != null) {
    --- End diff --
    
    same as for NIO, lets set serverCnxnClassCtr to NettyServerCnxn by default


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92274772
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    --- End diff --
    
    We usually put 1 minute (60000 ms) timeout, why decrease here? If **I** understood correctly, test timeouts define a maximum execution threshold, but they won't execute faster if we decrease, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86691087
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    +            close();
    --- End diff --
    
    I agree that it must be closing somewhere - just need to trace where


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92297538
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIONoException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  private void testZKNoExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "NoException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(path, false);
    +      if ( stats != null ) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.error("ConnectionLossException: {}", cle);
    +      fail("No exception should be thrown");
    +    } catch (Throwable t) {
    +      // error
    +      LOG.error("Throwable {}", t);
    +      fail("No exception should be thrown");
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +  @Test (timeout = 10000)
    --- End diff --
    
    Why 10000 and not 60000?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90553827
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -165,31 +163,35 @@ public void process(WatchedEvent event) {
         @Override
         public void sendResponse(ReplyHeader h, Record r, String tag)
                 throws IOException {
    -        if (!channel.isOpen()) {
    -            return;
    -        }
    -        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    -        // Make space for length
    -        BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
             try {
    -            baos.write(fourBytes);
    -            bos.writeRecord(h, "header");
    -            if (r != null) {
    -                bos.writeRecord(r, tag);
    +            if (!channel.isOpen()) {
    +                return;
                 }
    -            baos.close();
    -        } catch (IOException e) {
    -            LOG.error("Error serializing response");
    -        }
    -        byte b[] = baos.toByteArray();
    -        ByteBuffer bb = ByteBuffer.wrap(b);
    -        bb.putInt(b.length - 4).rewind();
    -        sendBuffer(bb);
    -        if (h.getXid() > 0) {
    -            // zks cannot be null otherwise we would not have gotten here!
    -            if (!zkServer.shouldThrottle(outstandingCount.decrementAndGet())) {
    -                enableRecv();
    +            ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +            // Make space for length
    +            BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
    +            try {
    +                baos.write(fourBytes);
    +                bos.writeRecord(h, "header");
    +                if (r != null) {
    +                    bos.writeRecord(r, tag);
    +                }
    +                baos.close();
    +            } catch (IOException e) {
    --- End diff --
    
    I did not modify this code - it was like that before, but potentially - yes it makes sense to rethrow
    I would say there are multiple places I came across where exceptions are swallowed 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #99: ZOOKEEPER-2549 Add exception handling to sendResponse

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/99
  
    +1 with latest update.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #99: ZOOKEEPER-2549 Add exception handling to sendResponse

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on the issue:

    https://github.com/apache/zookeeper/pull/99
  
    will do tomorrow. Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90783210
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    Take my words back. I need it to be public as I use reflection to create it in NettyServerCnxnFactory and if it is not I would have to do couple of more steps during init to set access to public which is unnecessary


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92282690
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    --- End diff --
    
    It is a good practice (even tough I dunno we use consistently on ZK codebase) that we use an idiom like the one below:
    
    ```
      private String previousProperty = null;
    
      @Before
      public void setUp() {
             previous = System.getProperty(MY_PROPERTY_NAME);
             System.setProperty(MY_PROPERTY_NAME, "new_value");
      }
    
     @After
     public void tearDown() {
          System.setProperty(MY_PROPERTY_NAME, previousValue);
     }
    ```
    
    This preserves the previous value of the ``System.property()``, **afaik**.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90782565
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    I think I did this to match NIOServerCnxn constructor. It can be kept package level, as my test class is in the same package namespace. I can change it back, but it will be inconsistent with NIO



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90924628
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    @yufeldman Thanks for explaining, makes sense to me. It is required to run the tests, however I do find another problem with tests: it looks like the Netty tests (testNetty*) never run with current configuration. Proof: remove the public access specifier appertain to `NettyServerCnxn` and all tests of `ServerCxnExceptionsTest` still pass. We expect Netty related tests fail here without public access specifier, right? Now put back the public for `NettyServerCnxn` but remove the public access specifier appertains to `NIOServercCnxn`, now all tests failed while we expect only NIO tests fail but Netty tests pass. 
    
    It's likely caused by the intervening of the java system properties that controls the Netty vs NIO server selection. One solution is to split the `ServerCxnExceptionsTest` into Netty and NIO specific tests. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92326091
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIONoException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  private void testZKNoExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "NoException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(path, false);
    +      if ( stats != null ) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.error("ConnectionLossException: {}", cle);
    +      fail("No exception should be thrown");
    +    } catch (Throwable t) {
    +      // error
    +      LOG.error("Throwable {}", t);
    +      fail("No exception should be thrown");
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +  @Test (timeout = 10000)
    +  public void testNettyRunTimeException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKRunTimeExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    --- End diff --
    
    see above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92297567
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIONoException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  private void testZKNoExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "NoException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(path, false);
    +      if ( stats != null ) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.error("ConnectionLossException: {}", cle);
    +      fail("No exception should be thrown");
    +    } catch (Throwable t) {
    +      // error
    +      LOG.error("Throwable {}", t);
    +      fail("No exception should be thrown");
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +  @Test (timeout = 10000)
    +  public void testNettyRunTimeException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKRunTimeExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    --- End diff --
    
    Why 10000 and not 60000?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86674332
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java ---
    @@ -842,7 +868,26 @@ private void addCnxn(NIOServerCnxn cnxn) {
     
         protected NIOServerCnxn createConnection(SocketChannel sock,
                 SelectionKey sk, SelectorThread selectorThread) throws IOException {
    -        return new NIOServerCnxn(zkServer, sock, sk, this, selectorThread);
    +
    +        NIOServerCnxn cnxn = null;
    +
    +        if (serverCnxnClassCtr != null) {
    --- End diff --
    
    Actually I tend to come back to double instantiation - as it will be used only in UnitTests (serverCnxnClassCtr != null), while manipulation with constructor (invocation of newInstance) instead of directly creating instance of NIOServerCnxn will be really a hot path 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by rgs1 <gi...@git.apache.org>.
Github user rgs1 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86672385
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java ---
    @@ -842,7 +868,26 @@ private void addCnxn(NIOServerCnxn cnxn) {
     
         protected NIOServerCnxn createConnection(SocketChannel sock,
                 SelectionKey sk, SelectorThread selectorThread) throws IOException {
    -        return new NIOServerCnxn(zkServer, sock, sk, this, selectorThread);
    +
    +        NIOServerCnxn cnxn = null;
    +
    +        if (serverCnxnClassCtr != null) {
    --- End diff --
    
    why not set serverCnxnClassCtr to NIOServerCnxn by default to avoid looking up the right constructor every time we want to create a new connection (a bit of a hot path)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92266393
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    --- End diff --
    
    ``exceptionType`` is never used!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92283709
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIONoException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  private void testZKNoExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "NoException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(path, false);
    +      if ( stats != null ) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.error("ConnectionLossException: {}", cle);
    +      fail("No exception should be thrown");
    +    } catch (Throwable t) {
    +      // error
    +      LOG.error("Throwable {}", t);
    +      fail("No exception should be thrown");
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +  @Test (timeout = 10000)
    +  public void testNettyRunTimeException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKRunTimeExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIORunTimeException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKRunTimeExceptionHelper();
    +  }
    +
    +  private void testZKRunTimeExceptionHelper() throws Exception {
    --- End diff --
    
    My gut feeling is that most -- if not all -- of those helpers can be collapsed into a single method. Or at least two, because there's a lot of code duplication in those helpers. For example, the exception type could be passed via parameter as well as a boolean (``shouldFail``) that states if the method could or not throw an exception. Just an idea, but this could save some repetition, I **guess**. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92274850
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    --- End diff --
    
    Same as above: why decrease the timeout?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92271380
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    --- End diff --
    
    1. Why we need to log the message here?
    
    2. Why we need to serialize ``cle``? Would it be ``cle.getMessage()``?
    
    3. Wouldn't be better to let it throw the ``KeeperException.ConnectionLossException`` and make the test catch it with ``expected`` as below?
    
    ```
    @Test(timeout = 60000, expected = KeeperException.ConnectionLossException)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90942133
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    @yufeldman Yeah the tests are fine. What I mentioned that leads to `NettyServerCnxnx` not get instantiated only happens in erroneous cases when `ZOOKEEPER_SERVER_CNXN_FACTORY` is not initialized properly - this happens when the public specifier was removed so all tests fall back to create `NIOServerCnxnx` instead. That's not a real alarm.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90549964
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    --- End diff --
    
    We're using LOG.debug, so it shouldn't be an issue on prod.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90762715
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -165,31 +163,35 @@ public void process(WatchedEvent event) {
         @Override
         public void sendResponse(ReplyHeader h, Record r, String tag)
                 throws IOException {
    -        if (!channel.isOpen()) {
    -            return;
    -        }
    -        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    -        // Make space for length
    -        BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
             try {
    -            baos.write(fourBytes);
    -            bos.writeRecord(h, "header");
    -            if (r != null) {
    -                bos.writeRecord(r, tag);
    +            if (!channel.isOpen()) {
    +                return;
                 }
    -            baos.close();
    -        } catch (IOException e) {
    -            LOG.error("Error serializing response");
    -        }
    -        byte b[] = baos.toByteArray();
    -        ByteBuffer bb = ByteBuffer.wrap(b);
    -        bb.putInt(b.length - 4).rewind();
    -        sendBuffer(bb);
    -        if (h.getXid() > 0) {
    -            // zks cannot be null otherwise we would not have gotten here!
    -            if (!zkServer.shouldThrottle(outstandingCount.decrementAndGet())) {
    -                enableRecv();
    +            ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +            // Make space for length
    +            BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
    +            try {
    +                baos.write(fourBytes);
    +                bos.writeRecord(h, "header");
    +                if (r != null) {
    +                    bos.writeRecord(r, tag);
    +                }
    +                baos.close();
    +            } catch (IOException e) {
    --- End diff --
    
    Agree on re-throw the exception here. We could just remove the try - catch block here given there is a new try - catch block added in this PR that wraps the entire `sendResponse.` 
    We can start triaging other places where the exceptions are swallowed but should re-throw after this PR merging in.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92296064
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MockNettyServerCnxn.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.server;
    +
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.proto.ReplyHeader;
    +import org.jboss.netty.channel.Channel;
    +
    +import java.io.IOException;
    +import java.lang.reflect.Field;
    +import java.lang.reflect.Modifier;
    +
    +/**
    + * Helper class to test different scenarios in NettyServerCnxn
    + */
    +public class MockNettyServerCnxn extends NettyServerCnxn {
    --- End diff --
    
    In this file, tab is indented with 2 spaces while the rest of the ZooKeeper files use 4 spaces (I only discovered this 'cause my IDE complained about it).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90763391
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    +            close();
    --- End diff --
    
    >> It was not closing (I think) before as exception was swallowed since sendResponse in NIOServerCnxn was not throwing IOException
    
    Yes, I think the connection was not closing before in cases of exception thrown from `NIOServerCnxn.sendResponse` which swallows everything. The change in this PR changes the behavior by closing the connection in case of exceptions occur in sendResponse. I am leaning towards the old behavior of NOT closing the connection, because the connection looks pretty innocent - in fact `NIOServerCnxn.sendResponse` does not involve any socket IO I believe, it just queuing stuff to be send over sockets. So if something goes wrong, we just do our best effort by logging what's wrong - rather than trying mess up with sockets which seems out of responsibilities of `NIOServerCnxn.sendResponse`. Similarly since `NIOServerCnxn.sendResponse` does not directly involve sockets, there should not be any leaks in case sendResponse screw up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92326026
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    --- End diff --
    
    why to keep it more than needed? 30 sec. is standard for ZK timeout, so 1 min would be overkill.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92269683
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    --- End diff --
    
    Method name doesn't follow camel case convention (``nioSetup`` instead of ``NIOSetup``).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92296029
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    --- End diff --
    
    In this file, tab is indented with 2 spaces while the rest of the ZooKeeper files use 4 spaces (I only discovered this 'cause my IDE complained about it).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #99: ZOOKEEPER-2549 Add exception handling to sendResponse

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/99
  
    @yufeldman could you please fix some coding style issues pointed out by @eribeiro (4 spaces indentation, extra white space, etc)? I'll merge your patch once those comments are addressed. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by yufeldman <gi...@git.apache.org>.
Github user yufeldman commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r86673997
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java ---
    @@ -716,7 +716,12 @@ public void process(WatchedEvent event) {
             // Convert WatchedEvent to a type that can be sent over the wire
             WatcherEvent e = event.getWrapper();
     
    -        sendResponse(h, e, "notification");
    +        try {
    +            sendResponse(h, e, "notification");
    +        } catch (IOException ex) {
    +            LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex);
    +            close();
    --- End diff --
    
    It was not closing (I think) before as exception was swallowed since sendResponse in NIOServerCnxn was not throwing IOException


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92297258
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIONoException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  private void testZKNoExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "NoException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(path, false);
    +      if ( stats != null ) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.error("ConnectionLossException: {}", cle);
    +      fail("No exception should be thrown");
    +    } catch (Throwable t) {
    +      // error
    +      LOG.error("Throwable {}", t);
    +      fail("No exception should be thrown");
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +  @Test (timeout = 10000)
    +  public void testNettyRunTimeException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKRunTimeExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIORunTimeException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKRunTimeExceptionHelper();
    +  }
    +
    +  private void testZKRunTimeExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "RunTimeException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      String returnPath = zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(returnPath, false);
    +      if ( stats != null ) {
    --- End diff --
    
    nit: extra spaces after the parenthesis.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r90960756
  
    --- Diff: src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java ---
    @@ -71,7 +71,7 @@
         NettyServerCnxnFactory factory;
         boolean initialized;
         
    -    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    +    public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
    --- End diff --
    
    >> It is not the case, they don't fall back to NIO - they fail.
    The fallback I was referring to is https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java#L130. It gets hit when public was removed from NIOServerCnxn for Netty* tests. An example call stack (note that a Netty test complaining about NIOServerCnxnFactory):
    ` INFO  [main:ZKTestCase$1@70] - FAILED testNettyRunTimeException
    java.io.IOException: Couldn't instantiate org.apache.zookeeper.server.NIOServerCnxnFactory
    >-------at org.apache.zookeeper.server.ServerCnxnFactory.createFactory(ServerCnxnFactory.java:142)
    >-------at org.apache.zookeeper.server.ServerCnxnFactory.createFactory(ServerCnxnFactory.java:158)
    >-------at org.apache.zookeeper.server.ServerCnxnFactory.createFactory(ServerCnxnFactory.java:152)`
    
    Thanks for persisting on this, but I don't think this erroneous case need to be investigated further as it would not happen when real test cases were running.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92296352
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    --- End diff --
    
    nit: it's nice to have an ordering on the definition of methods: public, protected, private. That is, it would be nice to move the private methods to the end of the file.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #99: ZOOKEEPER-2549 Add exception handling to sendRes...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/99#discussion_r92297351
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ServerCxnExceptionsTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.data.Stat;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.AfterClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static org.junit.Assert.fail;
    +
    +/**
    + * Unit tests to test different exceptions scenarious in sendResponse
    + */
    +public class ServerCxnExceptionsTest extends ClientBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(ServerCxnExceptionsTest.class);
    +
    +  private String exceptionType;
    +
    +  private void NettySetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NettyServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNettyServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  private void NIOSetup() throws Exception {
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
    +      "org.apache.zookeeper.server.NIOServerCnxnFactory");
    +    System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN, "org.apache.zookeeper.server.MockNIOServerCnxn");
    +    System.setProperty("exception.type", "NoException");
    +  }
    +
    +  @AfterClass
    +  public static void tearDownAfterClass() throws Exception {
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
    +    System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN);
    +    System.clearProperty("exception.type");
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNettyIOException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  @Test (timeout = 60000)
    +  public void testNIOIOException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testIOExceptionHelper();
    +  }
    +
    +  private void testIOExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "IOException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      fail("Should not come here");
    +      Stat stats = zk.exists(path, false);
    +      if (stats != null) {
    +        int length = stats.getDataLength();
    +      }
    +    } catch (KeeperException.ConnectionLossException cle) {
    +      LOG.info("ConnectionLossException: {}", cle);
    +    } finally {
    +      zk.close();
    +    }
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNettyNoException() throws Exception {
    +    tearDown();
    +    NettySetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  @Test (timeout = 10000)
    +  public void testNIONoException() throws Exception {
    +    tearDown();
    +    NIOSetup();
    +    testZKNoExceptionHelper();
    +  }
    +
    +  private void testZKNoExceptionHelper() throws Exception {
    +    System.setProperty("exception.type", "NoException");
    +    super.setUp();
    +    final ZooKeeper zk = createClient();
    +    final String path = "/a";
    +    try {
    +      // make sure zkclient works
    +      zk.create(path, "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +        CreateMode.EPHEMERAL);
    +      Stat stats = zk.exists(path, false);
    +      if ( stats != null ) {
    --- End diff --
    
    nit: extra spaces after the parenthesis.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---