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 2018/04/24 23:46:27 UTC

zookeeper git commit: ZOOKEEPER-2415: SessionTest is using Thread deprecated API. (branch-3.4)

Repository: zookeeper
Updated Branches:
  refs/heads/branch-3.4 e5259e437 -> a188308cf


ZOOKEEPER-2415: SessionTest is using Thread deprecated API. (branch-3.4)

This is the 3.4 version of https://github.com/apache/zookeeper/pull/497
It also include the following patch which was a dependency of the 3.5 version:
https://github.com/apache/zookeeper/commit/2b8f98be7407da4417dd192f3de4605a99e3cd15

Author: Michi Mutsuzaki <mi...@apache.org>
Author: Andor Molnar <an...@cloudera.com>

Reviewers: phunt@apache.org

Closes #506 from anmolnar/ZOOKEEPER-2415_24 and squashes the following commits:

900a7dc14 [Andor Molnar] ZOOKEEPER-2415. Refactor testSessionTimeout() to live in separate class and not to use deprecated API. Also improved performance.
58a228bd4 [Michi Mutsuzaki] ZOOKEEPER-1730. Make ZooKeeper easier to test - support simulating a session expiration (Jordan Zimmerman via michim)

Change-Id: Ie28ed0e256883d214b6ccef4baa06ece8b568d4f


Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/a188308c
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/a188308c
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/a188308c

Branch: refs/heads/branch-3.4
Commit: a188308cfb96da62dcd212b53247177f62771966
Parents: e5259e4
Author: Andor Molnar <an...@cloudera.com>
Authored: Tue Apr 24 16:45:33 2018 -0700
Committer: Patrick Hunt <ph...@apache.org>
Committed: Tue Apr 24 16:45:33 2018 -0700

----------------------------------------------------------------------
 .../main/org/apache/zookeeper/ClientCnxn.java   |   3 +-
 .../main/org/apache/zookeeper/Testable.java     |  29 +++++
 .../main/org/apache/zookeeper/ZooKeeper.java    |   6 +
 .../org/apache/zookeeper/ZooKeeperTestable.java |  47 +++++++
 .../org/apache/zookeeper/TestableZooKeeper.java |   9 ++
 .../org/apache/zookeeper/test/SessionTest.java  |  66 ----------
 .../zookeeper/test/SessionTimeoutTest.java      | 129 +++++++++++++++++++
 7 files changed, 222 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/a188308c/src/java/main/org/apache/zookeeper/ClientCnxn.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java
index 64b8810..390c363 100644
--- a/src/java/main/org/apache/zookeeper/ClientCnxn.java
+++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java
@@ -1388,7 +1388,8 @@ public class ClientCnxn {
 
     private int xid = 1;
 
-    private volatile States state = States.NOT_CONNECTED;
+    // @VisibleForTesting
+    volatile States state = States.NOT_CONNECTED;
 
     /*
      * getXid() is called externally by ClientCnxnNIO::doIO() when packets are sent from the outgoingQueue to

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/a188308c/src/java/main/org/apache/zookeeper/Testable.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/Testable.java b/src/java/main/org/apache/zookeeper/Testable.java
new file mode 100644
index 0000000..b38d2c3
--- /dev/null
+++ b/src/java/main/org/apache/zookeeper/Testable.java
@@ -0,0 +1,29 @@
+/**
+ * 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;
+
+/**
+ * Abstraction that exposes various methods useful for testing ZooKeeper
+ */
+public interface Testable {
+    /**
+     * Cause the ZooKeeper instance to behave as if the session expired
+     */
+    void injectSessionExpiration();
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/a188308c/src/java/main/org/apache/zookeeper/ZooKeeper.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java
index 482e135..3e46ee7 100644
--- a/src/java/main/org/apache/zookeeper/ZooKeeper.java
+++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java
@@ -97,6 +97,7 @@ public class ZooKeeper {
         Environment.logEnv("Client environment:", LOG);
     }
 
+
     public ZooKeeperSaslClient getSaslClient() {
         return cnxn.zooKeeperSaslClient;
     }
@@ -595,6 +596,11 @@ public class ZooKeeper {
         cnxn.start();
     }
 
+    // VisibleForTesting
+    public Testable getTestable() {
+        return new ZooKeeperTestable(this, cnxn);
+    }
+
     /**
      * The session id for this ZooKeeper client instance. The value returned is
      * not valid until the client connects to a server and may change after a

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/a188308c/src/java/main/org/apache/zookeeper/ZooKeeperTestable.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ZooKeeperTestable.java b/src/java/main/org/apache/zookeeper/ZooKeeperTestable.java
new file mode 100644
index 0000000..775d1a2
--- /dev/null
+++ b/src/java/main/org/apache/zookeeper/ZooKeeperTestable.java
@@ -0,0 +1,47 @@
+/**
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ZooKeeperTestable implements Testable {
+    private static final Logger LOG = LoggerFactory
+            .getLogger(ZooKeeperTestable.class);
+
+    private final ZooKeeper zooKeeper;
+    private final ClientCnxn clientCnxn;
+
+    ZooKeeperTestable(ZooKeeper zooKeeper, ClientCnxn clientCnxn) {
+        this.zooKeeper = zooKeeper;
+        this.clientCnxn = clientCnxn;
+    }
+
+    @Override
+    public void injectSessionExpiration() {
+        LOG.info("injectSessionExpiration() called");
+
+        clientCnxn.eventThread.queueEvent(new WatchedEvent(
+                Watcher.Event.EventType.None,
+                Watcher.Event.KeeperState.Expired, null));
+        clientCnxn.eventThread.queueEventOfDeath();
+        clientCnxn.sendThread.getClientCnxnSocket().wakeupCnxn();
+        clientCnxn.state = ZooKeeper.States.CLOSED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/a188308c/src/java/test/org/apache/zookeeper/TestableZooKeeper.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/TestableZooKeeper.java b/src/java/test/org/apache/zookeeper/TestableZooKeeper.java
index 4d46fdf..dd70056 100644
--- a/src/java/test/org/apache/zookeeper/TestableZooKeeper.java
+++ b/src/java/test/org/apache/zookeeper/TestableZooKeeper.java
@@ -121,4 +121,13 @@ public class TestableZooKeeper extends ZooKeeper {
             Record response, WatchRegistration watchRegistration) throws InterruptedException {
         return cnxn.submitRequest(h, request, response, watchRegistration);
     }
+
+    /** Testing only!!! Really!!!! This is only here to test when the client
+     * disconnects from the server w/o sending a session disconnect (ie
+     * ending the session cleanly). The server will eventually notice the
+     * client is no longer pinging and will timeout the session.
+     */
+    public void disconnect() {
+        cnxn.disconnect();
+    }
 }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/a188308c/src/java/test/org/apache/zookeeper/test/SessionTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/SessionTest.java b/src/java/test/org/apache/zookeeper/test/SessionTest.java
index 0740ebb..891b0f0 100644
--- a/src/java/test/org/apache/zookeeper/test/SessionTest.java
+++ b/src/java/test/org/apache/zookeeper/test/SessionTest.java
@@ -22,7 +22,6 @@ import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
@@ -236,71 +235,6 @@ public class SessionTest extends ZKTestCase {
         Assert.assertEquals(KeeperException.Code.SESSIONEXPIRED.toString(), cb.toString());        
     }
 
-    private List<Thread> findThreads(String name) {
-        int threadCount = Thread.activeCount();
-        Thread threads[] = new Thread[threadCount*2];
-        threadCount = Thread.enumerate(threads);
-        ArrayList<Thread> list = new ArrayList<Thread>();
-        for(int i = 0; i < threadCount; i++) {
-            if (threads[i].getName().indexOf(name) != -1) {
-                list.add(threads[i]);
-            }
-        }
-        return list;
-    }
-
-    /**
-     * Make sure ephemerals get cleaned up when a session times out.
-     */
-    @Test
-    public void testSessionTimeout() throws Exception {
-        final int TIMEOUT = 5000;
-        List<Thread> etBefore = findThreads("EventThread");
-        List<Thread> stBefore = findThreads("SendThread");
-        DisconnectableZooKeeper zk = createClient(TIMEOUT);
-        zk.create("/stest", new byte[0], Ids.OPEN_ACL_UNSAFE,
-                CreateMode.EPHEMERAL);
-
-        // Find the new event and send threads
-        List<Thread> etAfter = findThreads("EventThread");
-        List<Thread> stAfter = findThreads("SendThread");
-        Thread eventThread = null;
-        Thread sendThread = null;
-        for(Thread t: etAfter) {
-            if (!etBefore.contains(t)) {
-                eventThread = t;
-                break;
-            }
-        }
-        for(Thread t: stAfter) {
-            if (!stBefore.contains(t)) {
-                sendThread = t;
-                break;
-            }
-        }
-        sendThread.suspend();
-        //zk.disconnect();
-
-        Thread.sleep(TIMEOUT*2);
-        sendThread.resume();
-        eventThread.join(TIMEOUT);
-        Assert.assertFalse("EventThread is still running", eventThread.isAlive());
-
-        zk = createClient(TIMEOUT);
-        zk.create("/stest", new byte[0], Ids.OPEN_ACL_UNSAFE,
-                CreateMode.EPHEMERAL);
-        tearDown();
-        zk.close();
-        zk.disconnect();
-        setUp();
-
-        zk = createClient(TIMEOUT);
-        Assert.assertTrue(zk.exists("/stest", false) != null);
-        Thread.sleep(TIMEOUT*2);
-        Assert.assertTrue(zk.exists("/stest", false) == null);
-        zk.close();
-    }
-
     /**
      * Make sure that we cannot have two connections with the same
      * session id.

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/a188308c/src/java/test/org/apache/zookeeper/test/SessionTimeoutTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/SessionTimeoutTest.java b/src/java/test/org/apache/zookeeper/test/SessionTimeoutTest.java
new file mode 100644
index 0000000..09badae
--- /dev/null
+++ b/src/java/test/org/apache/zookeeper/test/SessionTimeoutTest.java
@@ -0,0 +1,129 @@
+/**
+ * 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.test;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.TestableZooKeeper;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class SessionTimeoutTest extends ClientBase {
+    protected static final Logger LOG = LoggerFactory.getLogger(SessionTimeoutTest.class);
+
+    private TestableZooKeeper zk;
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        zk = createClient();
+    }
+
+    @Test
+    public void testSessionExpiration() throws InterruptedException,
+            KeeperException {
+        final CountDownLatch expirationLatch = new CountDownLatch(1);
+        Watcher watcher = new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                if ( event.getState() == Event.KeeperState.Expired ) {
+                    expirationLatch.countDown();
+                }
+            }
+        };
+        zk.exists("/foo", watcher);
+
+        zk.getTestable().injectSessionExpiration();
+        Assert.assertTrue(expirationLatch.await(5, TimeUnit.SECONDS));
+
+        boolean gotException = false;
+        try {
+            zk.exists("/foo", false);
+            Assert.fail("Should have thrown a SessionExpiredException");
+        } catch (KeeperException.SessionExpiredException e) {
+            // correct
+            gotException = true;
+        }
+        Assert.assertTrue(gotException);
+    }
+
+    /**
+     * Make sure ephemerals get cleaned up when session disconnects.
+     */
+    @Test
+    public void testSessionDisconnect() throws KeeperException, InterruptedException, IOException {
+        zk.create("/sdisconnect", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                CreateMode.EPHEMERAL);
+        assertNotNull("Ephemeral node has not been created", zk.exists("/sdisconnect", null));
+
+        zk.close();
+
+        zk = createClient();
+        assertNull("Ephemeral node shouldn't exist after client disconnect", zk.exists("/sdisconnect", null));
+    }
+
+    /**
+     * Make sure ephemerals are kept when session restores.
+     */
+    @Test
+    public void testSessionRestore() throws KeeperException, InterruptedException, IOException {
+        zk.create("/srestore", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                CreateMode.EPHEMERAL);
+        assertNotNull("Ephemeral node has not been created", zk.exists("/srestore", null));
+
+        zk.disconnect();
+        zk.close();
+
+        zk = createClient();
+        assertNotNull("Ephemeral node should be present when session is restored", zk.exists("/srestore", null));
+    }
+
+    /**
+     * Make sure ephemerals are kept when server restarts.
+     */
+    @Test
+    public void testSessionSurviveServerRestart() throws Exception {
+        zk.create("/sdeath", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                CreateMode.EPHEMERAL);
+        assertNotNull("Ephemeral node has not been created", zk.exists("/sdeath", null));
+
+        zk.disconnect();
+        stopServer();
+        startServer();
+        zk = createClient();
+
+        assertNotNull("Ephemeral node should be present when server restarted", zk.exists("/sdeath", null));
+    }
+}