You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by js...@apache.org on 2016/04/07 23:03:11 UTC

aurora git commit: Generalize ZooKeeper Test Utils.

Repository: aurora
Updated Branches:
  refs/heads/master 915459dac -> c0cb631de


Generalize ZooKeeper Test Utils.

This makes way for Curator client based in-process integration tests.

Bugs closed: AURORA-1468

Reviewed at https://reviews.apache.org/r/45886/


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

Branch: refs/heads/master
Commit: c0cb631de606f89dca9c002623d4bc96b1f83e53
Parents: 915459d
Author: John Sirois <js...@apache.org>
Authored: Thu Apr 7 15:03:08 2016 -0600
Committer: John Sirois <jo...@gmail.com>
Committed: Thu Apr 7 15:03:08 2016 -0600

----------------------------------------------------------------------
 .../testing/BaseZooKeeperClientTest.java        | 139 +++++++++++++++++++
 .../zookeeper/testing/BaseZooKeeperTest.java    | 110 +--------------
 .../zookeeper/testing/ZooKeeperTestServer.java  |  99 ++-----------
 .../common/zookeeper/CandidateImplTest.java     |   4 +-
 .../aurora/common/zookeeper/GroupTest.java      |   4 +-
 .../common/zookeeper/ServerSetImplTest.java     |   4 +-
 .../zookeeper/SingletonServiceImplTest.java     |   4 +-
 .../common/zookeeper/ZooKeeperClientTest.java   |   4 +-
 .../common/zookeeper/ZooKeeperUtilsTest.java    |   4 +-
 .../guice/client/ZooKeeperClientModule.java     |  15 +-
 .../aurora/scheduler/app/SchedulerIT.java       |   4 +-
 11 files changed, 174 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperClientTest.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperClientTest.java b/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperClientTest.java
new file mode 100644
index 0000000..b9eaedb
--- /dev/null
+++ b/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperClientTest.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed 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.aurora.common.zookeeper.testing;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.common.quantity.Time;
+import org.apache.aurora.common.zookeeper.ZooKeeperClient;
+
+/**
+ * A base-class for tests that interact with ZooKeeper via the commons ZooKeeperClient.
+ */
+public abstract class BaseZooKeeperClientTest extends BaseZooKeeperTest {
+
+  private final Amount<Integer, Time> defaultSessionTimeout;
+
+  /**
+   * Creates a test case where the test server uses its
+   * {@link ZooKeeperTestServer#DEFAULT_SESSION_TIMEOUT} for clients created without an explicit
+   * session timeout.
+   */
+  public BaseZooKeeperClientTest() {
+    this(ZooKeeperTestServer.DEFAULT_SESSION_TIMEOUT);
+  }
+
+  /**
+   * Creates a test case where the test server uses the given {@code defaultSessionTimeout} for
+   * clients created without an explicit session timeout.
+   */
+  public BaseZooKeeperClientTest(Amount<Integer, Time> defaultSessionTimeout) {
+    this.defaultSessionTimeout = Preconditions.checkNotNull(defaultSessionTimeout);
+  }
+
+
+  /**
+   * Starts zookeeper back up on the last used port.
+   */
+  protected final void restartNetwork() throws IOException, InterruptedException {
+    getServer().restartNetwork();
+  }
+
+  /**
+   * Shuts down the in-process zookeeper network server.
+   */
+  protected final void shutdownNetwork() {
+    getServer().shutdownNetwork();
+  }
+
+  /**
+   * Expires the active session for the given client.  The client should be one returned from
+   * {@link #createZkClient}.
+   *
+   * @param zkClient the client to expire
+   * @throws ZooKeeperClient.ZooKeeperConnectionException if a problem is encountered connecting to
+   *    the local zk server while trying to expire the session
+   * @throws InterruptedException if interrupted while requesting expiration
+   */
+  protected final void expireSession(ZooKeeperClient zkClient)
+      throws ZooKeeperClient.ZooKeeperConnectionException, InterruptedException {
+    getServer().expireClientSession(zkClient.get().getSessionId());
+  }
+
+  /**
+   * Returns the current port to connect to the in-process zookeeper instance.
+   */
+  protected final int getPort() {
+    return getServer().getPort();
+  }
+
+  /**
+   * Returns a new unauthenticated zookeeper client connected to the in-process zookeeper server
+   * with the default session timeout.
+   */
+  protected final ZooKeeperClient createZkClient() {
+    return createZkClient(ZooKeeperClient.Credentials.NONE);
+  }
+
+  /**
+   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
+   * the default session timeout.
+   */
+  protected final ZooKeeperClient createZkClient(ZooKeeperClient.Credentials credentials) {
+    return createZkClient(defaultSessionTimeout, credentials, Optional.absent());
+  }
+
+  /**
+   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
+   * the default session timeout.  The client is authenticated in the digest authentication scheme
+   * with the given {@code username} and {@code password}.
+   */
+  protected final ZooKeeperClient createZkClient(String username, String password) {
+    return createZkClient(ZooKeeperClient.digestCredentials(username, password));
+  }
+
+  /**
+   * Returns a new unauthenticated zookeeper client connected to the in-process zookeeper server
+   * with a custom {@code sessionTimeout}.
+   */
+  protected final ZooKeeperClient createZkClient(Amount<Integer, Time> sessionTimeout) {
+    return createZkClient(sessionTimeout, ZooKeeperClient.Credentials.NONE, Optional.absent());
+  }
+
+  /**
+   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
+   * the default session timeout and the custom chroot path.
+   */
+  protected final ZooKeeperClient createZkClient(String chrootPath) {
+    return createZkClient(defaultSessionTimeout, ZooKeeperClient.Credentials.NONE,
+        Optional.of(chrootPath));
+  }
+
+  private ZooKeeperClient createZkClient(
+      Amount<Integer, Time> sessionTimeout,
+      ZooKeeperClient.Credentials credentials,
+      Optional<String> chrootPath) {
+
+    ZooKeeperClient client = new ZooKeeperClient(sessionTimeout, credentials, chrootPath,
+        ImmutableList.of(InetSocketAddress.createUnresolved("127.0.0.1", getPort())));
+    addTearDown(client::close);
+    return client;
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperTest.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperTest.java b/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperTest.java
index 3242035..0e68987 100644
--- a/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperTest.java
+++ b/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/BaseZooKeeperTest.java
@@ -13,130 +13,34 @@
  */
 package org.apache.aurora.common.zookeeper.testing;
 
-import java.io.IOException;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.testing.TearDownTestCase;
-import org.apache.aurora.common.zookeeper.ZooKeeperClient;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 
 /**
- * A baseclass for in-process zookeeper tests.
- * Uses ZooKeeperTestHelper to start the server and create clients: new tests should directly use
- * that helper class instead of extending this class.
+ * A base-class for in-process zookeeper tests.
  */
 public abstract class BaseZooKeeperTest extends TearDownTestCase {
 
-  private final Amount<Integer, Time> defaultSessionTimeout;
   private ZooKeeperTestServer zkTestServer;
+
   @Rule
   public TemporaryFolder tmpFolder = new TemporaryFolder();
 
-  /**
-   * Creates a test case where the test server uses its
-   * {@link ZooKeeperTestServer#DEFAULT_SESSION_TIMEOUT} for clients created without an explicit
-   * session timeout.
-   */
-  public BaseZooKeeperTest() {
-    this(ZooKeeperTestServer.DEFAULT_SESSION_TIMEOUT);
-  }
-
-  /**
-   * Creates a test case where the test server uses the given {@code defaultSessionTimeout} for
-   * clients created without an explicit session timeout.
-   */
-  public BaseZooKeeperTest(Amount<Integer, Time> defaultSessionTimeout) {
-    this.defaultSessionTimeout = Preconditions.checkNotNull(defaultSessionTimeout);
-  }
-
   @Before
   public final void setUp() throws Exception {
-    zkTestServer = new ZooKeeperTestServer(
-        defaultSessionTimeout,
-        tmpFolder.newFolder(),
-        tmpFolder.newFolder());
+    zkTestServer = new ZooKeeperTestServer(tmpFolder.newFolder(), tmpFolder.newFolder());
     addTearDown(zkTestServer::stop);
     zkTestServer.startNetwork();
   }
 
   /**
-   * Starts zookeeper back up on the last used port.
-   */
-  protected final void restartNetwork() throws IOException, InterruptedException {
-    zkTestServer.restartNetwork();
-  }
-
-  /**
-   * Shuts down the in-process zookeeper network server.
-   */
-  protected final void shutdownNetwork() {
-    zkTestServer.shutdownNetwork();
-  }
-
-  /**
-   * Expires the active session for the given client.  The client should be one returned from
-   * {@link #createZkClient}.
+   * Returns the running in-process ZooKeeper server.
    *
-   * @param zkClient the client to expire
-   * @throws ZooKeeperClient.ZooKeeperConnectionException if a problem is encountered connecting to
-   *    the local zk server while trying to expire the session
-   * @throws InterruptedException if interrupted while requesting expiration
-   */
-  protected final void expireSession(ZooKeeperClient zkClient)
-      throws ZooKeeperClient.ZooKeeperConnectionException, InterruptedException {
-    zkTestServer.expireClientSession(zkClient);
-  }
-
-  /**
-   * Returns the current port to connect to the in-process zookeeper instance.
-   */
-  protected final int getPort() {
-    return zkTestServer.getPort();
-  }
-
-  /**
-   * Returns a new unauthenticated zookeeper client connected to the in-process zookeeper server
-   * with the default session timeout.
-   */
-  protected final ZooKeeperClient createZkClient() {
-    return zkTestServer.createClient();
-  }
-
-  /**
-   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
-   * the default session timeout.
-   */
-  protected final ZooKeeperClient createZkClient(ZooKeeperClient.Credentials credentials) {
-    return zkTestServer.createClient(credentials);
-  }
-
-  /**
-   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
-   * the default session timeout.  The client is authenticated in the digest authentication scheme
-   * with the given {@code username} and {@code password}.
-   */
-  protected final ZooKeeperClient createZkClient(String username, String password) {
-    return createZkClient(ZooKeeperClient.digestCredentials(username, password));
-  }
-
-  /**
-   * Returns a new unauthenticated zookeeper client connected to the in-process zookeeper server
-   * with a custom {@code sessionTimeout}.
-   */
-  protected final ZooKeeperClient createZkClient(Amount<Integer, Time> sessionTimeout) {
-    return zkTestServer.createClient(sessionTimeout);
-  }
-
-  /**
-   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
-   * the default session timeout and the custom chroot path.
+   * @return The in-process ZooKeeper server.
    */
-  protected final ZooKeeperClient createZkClient(String chrootPath) {
-    return zkTestServer.createClient(chrootPath);
+  protected final ZooKeeperTestServer getServer() {
+    return zkTestServer;
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java b/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java
index dc4d389..0ab24fa 100644
--- a/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java
+++ b/commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java
@@ -16,16 +16,11 @@ package org.apache.aurora.common.zookeeper.testing;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.Arrays;
-import java.util.LinkedList;
 
-import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Time;
-import org.apache.aurora.common.zookeeper.ZooKeeperClient;
 import org.apache.zookeeper.server.NIOServerCnxnFactory;
 import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
@@ -39,29 +34,16 @@ import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
  */
 public class ZooKeeperTestServer {
 
-  public static final Amount<Integer, Time> DEFAULT_SESSION_TIMEOUT =
-      Amount.of(100, Time.MILLISECONDS);
+  static final Amount<Integer, Time> DEFAULT_SESSION_TIMEOUT = Amount.of(100, Time.MILLISECONDS);
 
-  private final LinkedList<Runnable> cleanupActions = Lists.newLinkedList();
   private final File dataDir;
   private final File snapDir;
-  private final Amount<Integer, Time> defaultSessionTimeout;
 
   private ZooKeeperServer zooKeeperServer;
   private ServerCnxnFactory connectionFactory;
   private int port;
 
-  /**
-   * @param defaultSessionTimeout the default session timeout for clients created with
-   *     {@link #createClient()}.
-   * @throws IOException if there was aproblem creating the server's database
-   */
-  public ZooKeeperTestServer(
-      Amount<Integer, Time> defaultSessionTimeout,
-      File dataDir,
-      File snapDir) throws IOException {
-
-    this.defaultSessionTimeout = Preconditions.checkNotNull(defaultSessionTimeout);
+  public ZooKeeperTestServer(File dataDir, File snapDir) {
     this.dataDir = Preconditions.checkNotNull(dataDir);
     this.snapDir = Preconditions.checkNotNull(snapDir);
   }
@@ -87,7 +69,6 @@ public class ZooKeeperTestServer {
         new InetSocketAddress(port),
         60 /* Semi-arbitrary, max 60 connections is the default used by NIOServerCnxnFactory */);
     connectionFactory.startup(zooKeeperServer);
-    cleanupActions.addFirst(this::shutdownNetwork);
     port = zooKeeperServer.getClientPort();
   }
 
@@ -95,16 +76,13 @@ public class ZooKeeperTestServer {
    * Stops the zookeeper server.
    */
   public void stop() {
-    for (Runnable cleanup : cleanupActions) {
-      cleanup.run();
-    }
-    cleanupActions.clear();
+    shutdownNetwork();
   }
 
   /**
    * Starts zookeeper back up on the last used port.
    */
-  public final void restartNetwork() throws IOException, InterruptedException {
+  final void restartNetwork() throws IOException, InterruptedException {
     checkEphemeralPortAssigned();
     Preconditions.checkState(connectionFactory == null);
     startNetwork();
@@ -113,7 +91,7 @@ public class ZooKeeperTestServer {
   /**
    * Shuts down the in-process zookeeper network server.
    */
-  public final void shutdownNetwork() {
+  final void shutdownNetwork() {
     if (connectionFactory != null) {
       connectionFactory.shutdown(); // Also shuts down zooKeeperServer.
       connectionFactory = null;
@@ -121,17 +99,12 @@ public class ZooKeeperTestServer {
   }
 
   /**
-   * Expires the active session for the given client.  The client should be one returned from
-   * {@link #createClient}.
+   * Expires the client session with the given {@code sessionId}.
    *
-   * @param zkClient the client to expire
-   * @throws ZooKeeperClient.ZooKeeperConnectionException if a problem is encountered connecting to
-   *    the local zk server while trying to expire the session
-   * @throws InterruptedException if interrupted while requesting expiration
+   * @param sessionId The id of the client session to expire.
    */
-  public final void expireClientSession(ZooKeeperClient zkClient)
-      throws ZooKeeperClient.ZooKeeperConnectionException, InterruptedException {
-    zooKeeperServer.closeSession(zkClient.get().getSessionId());
+  void expireClientSession(long sessionId) {
+    zooKeeperServer.closeSession(sessionId);
   }
 
   /**
@@ -142,60 +115,6 @@ public class ZooKeeperTestServer {
     return port;
   }
 
-  /**
-   * Returns a new unauthenticated zookeeper client connected to the in-process zookeeper server
-   * with the default session timeout.
-   */
-  public final ZooKeeperClient createClient() {
-    return createClient(defaultSessionTimeout);
-  }
-
-  /**
-   * Returns a new unauthenticated zookeeper client connected to the in-process zookeeper server
-   * with the default session timeout and a custom {@code chrootPath}.
-   */
-  public final ZooKeeperClient createClient(String chrootPath) {
-    return createClient(defaultSessionTimeout, ZooKeeperClient.Credentials.NONE, Optional.of(chrootPath));
-  }
-
-  /**
-   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
-   * the default session timeout.
-   */
-  public final ZooKeeperClient createClient(ZooKeeperClient.Credentials credentials) {
-    return createClient(defaultSessionTimeout, credentials, Optional.<String>absent());
-  }
-
-  /**
-   * Returns a new unauthenticated zookeeper client connected to the in-process zookeeper server
-   * with a custom {@code sessionTimeout}.
-   */
-  public final ZooKeeperClient createClient(Amount<Integer, Time> sessionTimeout) {
-    return createClient(sessionTimeout, ZooKeeperClient.Credentials.NONE, Optional.<String>absent());
-  }
-
-  /**
-   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
-   * a custom {@code sessionTimeout}.
-   */
-  public final ZooKeeperClient createClient(Amount<Integer, Time> sessionTimeout,
-      ZooKeeperClient.Credentials credentials) {
-        return createClient(sessionTimeout, credentials, Optional.<String>absent());
-      }
-
-  /**
-   * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
-   * a custom {@code sessionTimeout} and a custom {@code chrootPath}.
-   */
-  public final ZooKeeperClient createClient(Amount<Integer, Time> sessionTimeout,
-      ZooKeeperClient.Credentials credentials, Optional<String> chrootPath) {
-
-    final ZooKeeperClient client = new ZooKeeperClient(sessionTimeout, credentials,
-        chrootPath, Arrays.asList(InetSocketAddress.createUnresolved("127.0.0.1", port)));
-    cleanupActions.addFirst(client::close);
-    return client;
-  }
-
   private void checkEphemeralPortAssigned() {
     Preconditions.checkState(port > 0, "startNetwork must be called first");
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/test/java/org/apache/aurora/common/zookeeper/CandidateImplTest.java
----------------------------------------------------------------------
diff --git a/commons/src/test/java/org/apache/aurora/common/zookeeper/CandidateImplTest.java b/commons/src/test/java/org/apache/aurora/common/zookeeper/CandidateImplTest.java
index 0763521..9c0cebe 100644
--- a/commons/src/test/java/org/apache/aurora/common/zookeeper/CandidateImplTest.java
+++ b/commons/src/test/java/org/apache/aurora/common/zookeeper/CandidateImplTest.java
@@ -24,7 +24,7 @@ import com.google.common.base.Supplier;
 import org.apache.aurora.common.base.ExceptionalCommand;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Time;
-import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
+import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperClientTest;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.junit.Before;
@@ -34,7 +34,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
-public class CandidateImplTest extends BaseZooKeeperTest {
+public class CandidateImplTest extends BaseZooKeeperClientTest {
   private static final List<ACL> ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;
   private static final String SERVICE = "/twitter/services/puffin_linkhose/leader";
   private static final Amount<Integer, Time> TIMEOUT = Amount.of(1, Time.MINUTES);

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/test/java/org/apache/aurora/common/zookeeper/GroupTest.java
----------------------------------------------------------------------
diff --git a/commons/src/test/java/org/apache/aurora/common/zookeeper/GroupTest.java b/commons/src/test/java/org/apache/aurora/common/zookeeper/GroupTest.java
index ac65442..9127b6e 100644
--- a/commons/src/test/java/org/apache/aurora/common/zookeeper/GroupTest.java
+++ b/commons/src/test/java/org/apache/aurora/common/zookeeper/GroupTest.java
@@ -30,7 +30,7 @@ import org.apache.aurora.common.zookeeper.Group.JoinException;
 import org.apache.aurora.common.zookeeper.Group.Membership;
 import org.apache.aurora.common.zookeeper.Group.NodeScheme;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient.Credentials;
-import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
+import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperClientTest;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.junit.Before;
 import org.junit.Test;
@@ -45,7 +45,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.fail;
 
-public class GroupTest extends BaseZooKeeperTest {
+public class GroupTest extends BaseZooKeeperClientTest {
 
   private ZooKeeperClient zkClient;
   private Group joinGroup;

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetImplTest.java
----------------------------------------------------------------------
diff --git a/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetImplTest.java b/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetImplTest.java
index 73049d8..f0c0cb4 100644
--- a/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetImplTest.java
+++ b/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetImplTest.java
@@ -30,7 +30,7 @@ import org.apache.aurora.common.thrift.Endpoint;
 import org.apache.aurora.common.thrift.ServiceInstance;
 import org.apache.aurora.common.thrift.Status;
 import org.apache.aurora.common.zookeeper.Group.JoinException;
-import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
+import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperClientTest;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
@@ -49,7 +49,7 @@ import static org.junit.Assert.fail;
  *
  * TODO(William Farner): Change this to remove thrift dependency.
  */
-public class ServerSetImplTest extends BaseZooKeeperTest {
+public class ServerSetImplTest extends BaseZooKeeperClientTest {
   private static final List<ACL> ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;
   private static final String SERVICE = "/twitter/services/puffin_hosebird";
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/test/java/org/apache/aurora/common/zookeeper/SingletonServiceImplTest.java
----------------------------------------------------------------------
diff --git a/commons/src/test/java/org/apache/aurora/common/zookeeper/SingletonServiceImplTest.java b/commons/src/test/java/org/apache/aurora/common/zookeeper/SingletonServiceImplTest.java
index 82df845..5f6cdd8 100644
--- a/commons/src/test/java/org/apache/aurora/common/zookeeper/SingletonServiceImplTest.java
+++ b/commons/src/test/java/org/apache/aurora/common/zookeeper/SingletonServiceImplTest.java
@@ -25,7 +25,7 @@ import org.apache.aurora.common.base.ExceptionalCommand;
 import org.apache.aurora.common.zookeeper.Candidate.Leader;
 import org.apache.aurora.common.zookeeper.SingletonService.LeaderControl;
 import org.apache.aurora.common.zookeeper.SingletonService.LeadershipListener;
-import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
+import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperClientTest;
 import org.easymock.Capture;
 import org.easymock.IExpectationSetters;
 import org.easymock.IMocksControl;
@@ -38,7 +38,7 @@ import static org.easymock.EasyMock.createControl;
 import static org.easymock.EasyMock.expect;
 import static org.junit.Assert.fail;
 
-public class SingletonServiceImplTest extends BaseZooKeeperTest {
+public class SingletonServiceImplTest extends BaseZooKeeperClientTest {
   private static final int PORT_A = 1234;
   private static final int PORT_B = 8080;
   private static final InetSocketAddress PRIMARY_ENDPOINT =

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperClientTest.java
----------------------------------------------------------------------
diff --git a/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperClientTest.java b/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperClientTest.java
index 1a2fb11..537d41e 100644
--- a/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperClientTest.java
+++ b/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperClientTest.java
@@ -22,7 +22,7 @@ import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient.Credentials;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient.ZooKeeperConnectionException;
-import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
+import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperClientTest;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException.ConnectionLossException;
 import org.apache.zookeeper.KeeperException.NoAuthException;
@@ -42,7 +42,7 @@ import static org.junit.Assert.fail;
 /**
  * @author John Sirois
  */
-public class ZooKeeperClientTest extends BaseZooKeeperTest {
+public class ZooKeeperClientTest extends BaseZooKeeperClientTest {
 
   public ZooKeeperClientTest() {
     super(Amount.of(1, Time.DAYS));

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperUtilsTest.java
----------------------------------------------------------------------
diff --git a/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperUtilsTest.java b/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperUtilsTest.java
index 32b937d..9e482a6 100644
--- a/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperUtilsTest.java
+++ b/commons/src/test/java/org/apache/aurora/common/zookeeper/ZooKeeperUtilsTest.java
@@ -15,7 +15,7 @@ package org.apache.aurora.common.zookeeper;
 
 import com.google.common.base.Charsets;
 
-import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
+import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperClientTest;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException.BadVersionException;
 import org.apache.zookeeper.KeeperException.NoAuthException;
@@ -34,7 +34,7 @@ import static org.junit.Assert.fail;
 /**
  * @author John Sirois
  */
-public class ZooKeeperUtilsTest extends BaseZooKeeperTest {
+public class ZooKeeperUtilsTest extends BaseZooKeeperClientTest {
   @Test
   public void testEnsurePath() throws Exception {
     ZooKeeperClient zkClient = createZkClient();

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule.java b/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule.java
index bf2111d..4239a86 100644
--- a/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule.java
@@ -73,15 +73,7 @@ public class ZooKeeperClientModule extends AbstractModule {
     Key<ZooKeeperClient> clientKey = keyFactory.create(ZooKeeperClient.class);
     if (config.inProcess) {
       File tempDir = Files.createTempDir();
-      try {
-        bind(ZooKeeperTestServer.class).toInstance(
-            new ZooKeeperTestServer(
-                ZooKeeperTestServer.DEFAULT_SESSION_TIMEOUT,
-                tempDir,
-                tempDir));
-      } catch (IOException e) {
-        throw Throwables.propagate(e);
-      }
+      bind(ZooKeeperTestServer.class).toInstance(new ZooKeeperTestServer(tempDir, tempDir));
 
       install(new PrivateModule() {
         @Override
@@ -144,7 +136,10 @@ public class ZooKeeperClientModule extends AbstractModule {
       } catch (IOException | InterruptedException e) {
         throw Throwables.propagate(e);
       }
-      return testServer.createClient(config.sessionTimeout, config.credentials);
+      return new ZooKeeperClient(
+          config.sessionTimeout,
+          config.credentials,
+          InetSocketAddress.createUnresolved("localhost", testServer.getPort()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/c0cb631d/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java b/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
index b449827..b1426bc 100644
--- a/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
@@ -47,7 +47,7 @@ import org.apache.aurora.common.stats.Stats;
 import org.apache.aurora.common.zookeeper.ServerSetImpl;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient.Credentials;
-import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
+import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperClientTest;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.gen.ScheduledTask;
 import org.apache.aurora.gen.ServerInfo;
@@ -106,7 +106,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-public class SchedulerIT extends BaseZooKeeperTest {
+public class SchedulerIT extends BaseZooKeeperClientTest {
 
   private static final Logger LOG = LoggerFactory.getLogger(SchedulerIT.class);