You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ph...@apache.org on 2011/11/02 22:54:01 UTC

svn commit: r1196820 - in /zookeeper/branches/branch-3.4: CHANGES.txt ivy.xml src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java src/java/test/org/apache/zookeeper/ClientReconnectTest.java

Author: phunt
Date: Wed Nov  2 21:54:01 2011
New Revision: 1196820

URL: http://svn.apache.org/viewvc?rev=1196820&view=rev
Log:
ZOOKEEPER-1271. testEarlyLeaderAbandonment failing on solaris - clients not retrying connection (mahadev via phunt)

Added:
    zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/ClientReconnectTest.java
Modified:
    zookeeper/branches/branch-3.4/CHANGES.txt
    zookeeper/branches/branch-3.4/ivy.xml
    zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java

Modified: zookeeper/branches/branch-3.4/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/CHANGES.txt?rev=1196820&r1=1196819&r2=1196820&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/CHANGES.txt (original)
+++ zookeeper/branches/branch-3.4/CHANGES.txt Wed Nov  2 21:54:01 2011
@@ -355,6 +355,9 @@ BUGFIXES: 
   
   ZOOKEEPER-1246. Dead code in PrepRequestProcessor catch Exception block. (camille)
 
+  ZOOKEEPER-1271. testEarlyLeaderAbandonment failing on solaris -
+  clients not retrying connection (mahadev via phunt)
+
 IMPROVEMENTS:
   ZOOKEEPER-724. Improve junit test integration - log harness information 
   (phunt via mahadev)

Modified: zookeeper/branches/branch-3.4/ivy.xml
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/ivy.xml?rev=1196820&r1=1196819&r2=1196820&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/ivy.xml (original)
+++ zookeeper/branches/branch-3.4/ivy.xml Wed Nov  2 21:54:01 2011
@@ -53,6 +53,8 @@
     <dependency org="org.vafer" name="jdeb" rev="0.8" conf="package->master"/>
 
     <dependency org="junit" name="junit" rev="4.8.1" conf="test->default"/>
+     <dependency org="org.mockito" name="mockito-all" rev="1.8.2"
+               conf="test->default"/>
     <dependency org="checkstyle" name="checkstyle" rev="5.0"
                 conf="test->default"/>
 

Modified: zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java?rev=1196820&r1=1196819&r2=1196820&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java (original)
+++ zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java Wed Nov  2 21:54:01 2011
@@ -176,23 +176,45 @@ public class ClientCnxnSocketNIO extends
             LOG.warn("Ignoring exception during selector close", e);
         }
     }
-
-    @Override
-    void connect(InetSocketAddress addr) throws IOException {
+    
+    /**
+     * create a socket channel.
+     * @return the created socket channel
+     * @throws IOException
+     */
+    SocketChannel createSock() throws IOException {
         SocketChannel sock;
         sock = SocketChannel.open();
         sock.configureBlocking(false);
         sock.socket().setSoLinger(false, -1);
         sock.socket().setTcpNoDelay(true);
+        return sock;
+    }
+
+    /**
+     * register with the selection and connect
+     * @param sock the {@link SocketChannel} 
+     * @param addr the address of remote host
+     * @throws IOException
+     */
+    void registerAndConnect(SocketChannel sock, InetSocketAddress addr) 
+    throws IOException {
+        sockKey = sock.register(selector, SelectionKey.OP_CONNECT);
+        boolean immediateConnect = sock.connect(addr);            
+        if (immediateConnect) {
+            sendThread.primeConnection();
+        }
+    }
+    
+    @Override
+    void connect(InetSocketAddress addr) throws IOException {
+        SocketChannel sock = createSock();
         try {
-            sockKey = sock.register(selector, SelectionKey.OP_CONNECT);
-            boolean immediateConnect = sock.connect(addr);            
-            if (immediateConnect) {
-                sendThread.primeConnection();
-            }
+           registerAndConnect(sock, addr);
         } catch (IOException e) {
             LOG.error("Unable to open socket to " + addr);
             sock.close();
+            throw e;
         }
         initialized = false;
 

Added: zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/ClientReconnectTest.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/ClientReconnectTest.java?rev=1196820&view=auto
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/ClientReconnectTest.java (added)
+++ zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/ClientReconnectTest.java Wed Nov  2 21:54:01 2011
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zookeeper;
+
+
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SocketChannel;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.zookeeper.client.HostProvider;
+import org.junit.Test;
+
+public class ClientReconnectTest extends TestCase {
+    private SocketChannel sc;
+    private CountDownLatch countDownLatch = new CountDownLatch(3);
+    
+    class MockCnxn extends ClientCnxnSocketNIO {
+        MockCnxn() throws IOException {
+            super();
+        }
+
+        @Override
+        void registerAndConnect(SocketChannel sock, InetSocketAddress addr) throws
+        IOException {
+            countDownLatch.countDown();
+            throw new IOException("failed to register");
+        }
+
+        @Override
+        SocketChannel createSock() {
+            return sc;
+        }
+    }
+
+    @Test
+    public void testClientReconnect() throws IOException, InterruptedException {
+        HostProvider hostProvider = mock(HostProvider.class);
+        when(hostProvider.size()).thenReturn(1);
+        InetSocketAddress inaddr = new InetSocketAddress(1111);
+        when(hostProvider.next(anyLong())).thenReturn(inaddr);
+        ZooKeeper zk = mock(ZooKeeper.class);
+        sc =  SocketChannel.open();
+
+        ClientCnxnSocketNIO nioCnxn = new MockCnxn();
+        ClientWatchManager watcher = mock(ClientWatchManager.class);
+        ClientCnxn clientCnxn = new ClientCnxn(
+                "tmp", hostProvider, 5000,
+                zk, watcher, nioCnxn, false);
+        clientCnxn.start();
+        countDownLatch.await(5000, TimeUnit.MILLISECONDS);
+        Assert.assertTrue(countDownLatch.getCount() == 0);
+        clientCnxn.close();
+    }
+}
\ No newline at end of file