You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2015/09/12 01:52:40 UTC

aurora git commit: Change ZooKeeperTestServer to not rely on ShutdownRegistry.

Repository: aurora
Updated Branches:
  refs/heads/master ec02a49df -> 5dccf92f4


Change ZooKeeperTestServer to not rely on ShutdownRegistry.

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


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

Branch: refs/heads/master
Commit: 5dccf92f479120d38dc0b2103cde86348c803985
Parents: ec02a49
Author: Bill Farner <wf...@apache.org>
Authored: Fri Sep 11 16:52:21 2015 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Fri Sep 11 16:52:36 2015 -0700

----------------------------------------------------------------------
 .../aurora/common/application/AppLauncher.java  |   1 -
 .../guice/client/ZooKeeperClientModule.java     | 173 ----------------
 .../client/flagged/FlaggedClientConfig.java     |  82 --------
 .../zookeeper/testing/BaseZooKeeperTest.java    |  19 +-
 .../zookeeper/testing/ZooKeeperTestServer.java  |  71 +++----
 config/legacy_untested_classes.txt              |   4 +
 .../apache/aurora/scheduler/app/AppModule.java  |   2 +-
 .../aurora/scheduler/app/SchedulerMain.java     |   6 +-
 .../log/mesos/MesosLogStreamModule.java         |   2 +-
 .../guice/client/ZooKeeperClientModule.java     | 200 +++++++++++++++++++
 .../client/flagged/FlaggedClientConfig.java     |  86 ++++++++
 .../aurora/scheduler/app/SchedulerIT.java       |   4 +-
 12 files changed, 332 insertions(+), 318 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/commons/src/main/java/org/apache/aurora/common/application/AppLauncher.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/application/AppLauncher.java b/commons/src/main/java/org/apache/aurora/common/application/AppLauncher.java
index 80b24bd..6496870 100644
--- a/commons/src/main/java/org/apache/aurora/common/application/AppLauncher.java
+++ b/commons/src/main/java/org/apache/aurora/common/application/AppLauncher.java
@@ -20,7 +20,6 @@ import java.util.logging.Logger;
 import com.google.common.base.Preconditions;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
-import com.google.inject.util.Modules;
 
 import org.apache.aurora.common.args.Arg;
 import org.apache.aurora.common.args.ArgFilters;

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/commons/src/main/java/org/apache/aurora/common/zookeeper/guice/client/ZooKeeperClientModule.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/zookeeper/guice/client/ZooKeeperClientModule.java b/commons/src/main/java/org/apache/aurora/common/zookeeper/guice/client/ZooKeeperClientModule.java
deleted file mode 100644
index b44e09e..0000000
--- a/commons/src/main/java/org/apache/aurora/common/zookeeper/guice/client/ZooKeeperClientModule.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/**
- * 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.guice.client;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.logging.Logger;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
-
-import com.google.inject.Inject;
-import com.google.inject.Key;
-import com.google.inject.PrivateModule;
-import com.google.inject.Provider;
-import com.google.inject.Singleton;
-
-import org.apache.aurora.common.application.ShutdownRegistry;
-import org.apache.aurora.common.inject.Bindings.KeyFactory;
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.common.quantity.Time;
-import org.apache.aurora.common.zookeeper.ZooKeeperClient;
-import org.apache.aurora.common.zookeeper.ZooKeeperClient.Credentials;
-import org.apache.aurora.common.zookeeper.ZooKeeperUtils;
-import org.apache.aurora.common.zookeeper.testing.ZooKeeperTestServer;
-
-/**
- * A guice binding module that configures and binds a {@link ZooKeeperClient} instance.
- */
-public class ZooKeeperClientModule extends PrivateModule {
-  private final KeyFactory keyFactory;
-  private final ClientConfig config;
-
-  /**
-   * Creates a new ZK client module from the provided configuration.
-   *
-   * @param config Configuration parameters for the client.
-   */
-  public ZooKeeperClientModule(ClientConfig config) {
-    this(KeyFactory.PLAIN, config);
-  }
-
-  /**
-   * Creates a new ZK client module from the provided configuration, using a key factory to
-   * qualify any bindings.
-   *
-   * @param keyFactory Factory to use when creating any exposed bindings.
-   * @param config Configuration parameters for the client.
-   */
-  public ZooKeeperClientModule(KeyFactory keyFactory, ClientConfig config) {
-    this.keyFactory = Preconditions.checkNotNull(keyFactory);
-    this.config = Preconditions.checkNotNull(config);
-  }
-
-  @Override
-  protected void configure() {
-    Key<ZooKeeperClient> clientKey = keyFactory.create(ZooKeeperClient.class);
-    if (config.inProcess) {
-      requireBinding(ShutdownRegistry.class);
-      // Bound privately to give the local provider access to configuration settings.
-      bind(ClientConfig.class).toInstance(config);
-      bind(clientKey).toProvider(LocalClientProvider.class).in(Singleton.class);
-    } else {
-      ZooKeeperClient client =
-          new ZooKeeperClient(config.sessionTimeout, config.credentials, config.chrootPath, config.servers);
-      bind(clientKey).toInstance(client);
-    }
-    expose(clientKey);
-  }
-
-  private static class LocalClientProvider implements Provider<ZooKeeperClient> {
-    private static final Logger LOG = Logger.getLogger(LocalClientProvider.class.getName());
-
-    private final ClientConfig config;
-    private final ShutdownRegistry shutdownRegistry;
-
-    @Inject
-    LocalClientProvider(ClientConfig config, ShutdownRegistry shutdownRegistry) {
-      this.config = Preconditions.checkNotNull(config);
-      this.shutdownRegistry = Preconditions.checkNotNull(shutdownRegistry);
-    }
-
-    @Override
-    public ZooKeeperClient get() {
-      ZooKeeperTestServer zooKeeperServer;
-      try {
-        zooKeeperServer = new ZooKeeperTestServer(0, shutdownRegistry);
-        zooKeeperServer.startNetwork();
-      } catch (IOException e) {
-        throw Throwables.propagate(e);
-      } catch (InterruptedException e) {
-        throw Throwables.propagate(e);
-      }
-
-      LOG.info("Embedded zookeeper cluster started on port " + zooKeeperServer.getPort());
-      return zooKeeperServer.createClient(config.sessionTimeout, config.credentials);
-    }
-  }
-
-  /**
-   * Composite type that contains configuration parameters used when creating a client.
-   * <p>
-   * Instances of this class are immutable, but builder-style chained calls are supported.
-   */
-  public static class ClientConfig {
-    public final Iterable<InetSocketAddress> servers;
-    public final boolean inProcess;
-    public final Amount<Integer, Time> sessionTimeout;
-    public final Optional<String> chrootPath;
-    public final Credentials credentials;
-
-    /**
-     * Creates a new client configuration.
-     *
-     * @param servers ZooKeeper server addresses.
-     * @param inProcess Whether to run and create clients for an in-process ZooKeeper server.
-     * @param chrootPath an optional chroot path
-     * @param sessionTimeout Timeout duration for established sessions.
-     * @param credentials ZooKeeper authentication credentials.
-     */
-    public ClientConfig(
-        Iterable<InetSocketAddress> servers,
-        Optional<String> chrootPath,
-        boolean inProcess,
-        Amount<Integer, Time> sessionTimeout,
-        Credentials credentials) {
-
-      this.servers = servers;
-      this.chrootPath = chrootPath;
-      this.inProcess = inProcess;
-      this.sessionTimeout = sessionTimeout;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Creates a new client configuration with defaults for the session timeout and credentials.
-     *
-     * @param servers ZooKeeper server addresses.
-     * @return A new configuration.
-     */
-    public static ClientConfig create(Iterable<InetSocketAddress> servers) {
-      return new ClientConfig(
-          servers,
-          Optional.<String> absent(),
-          false,
-          ZooKeeperUtils.DEFAULT_ZK_SESSION_TIMEOUT,
-          Credentials.NONE);
-    }
-
-    /**
-     * Creates a new configuration identical to this configuration, but with the provided
-     * credentials.
-     *
-     * @param credentials ZooKeeper authentication credentials.
-     * @return A modified clone of this configuration.
-     */
-    public ClientConfig withCredentials(Credentials credentials) {
-      return new ClientConfig(servers, chrootPath, inProcess, sessionTimeout, credentials);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/commons/src/main/java/org/apache/aurora/common/zookeeper/guice/client/flagged/FlaggedClientConfig.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/zookeeper/guice/client/flagged/FlaggedClientConfig.java b/commons/src/main/java/org/apache/aurora/common/zookeeper/guice/client/flagged/FlaggedClientConfig.java
deleted file mode 100644
index f3e3a84..0000000
--- a/commons/src/main/java/org/apache/aurora/common/zookeeper/guice/client/flagged/FlaggedClientConfig.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * 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.guice.client.flagged;
-
-import java.net.InetSocketAddress;
-import java.util.List;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Splitter;
-import com.google.common.collect.ImmutableList;
-
-import org.apache.aurora.common.args.Arg;
-import org.apache.aurora.common.args.CmdLine;
-import org.apache.aurora.common.args.constraints.NotEmpty;
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.common.quantity.Time;
-import org.apache.aurora.common.zookeeper.ZooKeeperClient;
-import org.apache.aurora.common.zookeeper.ZooKeeperClient.Credentials;
-import org.apache.aurora.common.zookeeper.ZooKeeperUtils;
-import org.apache.aurora.common.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
-
-/**
- * A factory that creates a {@link ClientConfig} instance based on command line argument values.
- */
-public class FlaggedClientConfig {
-  @CmdLine(name = "zk_in_proc",
-      help = "Launches an embedded zookeeper server for local testing causing -zk_endpoints "
-          + "to be ignored if specified.")
-  private static final Arg<Boolean> IN_PROCESS = Arg.create(false);
-
-  @NotEmpty
-  @CmdLine(name = "zk_endpoints", help ="Endpoint specification for the ZooKeeper servers.")
-  private static final Arg<List<InetSocketAddress>> ZK_ENDPOINTS = Arg.create();
-
-  @CmdLine(name = "zk_chroot_path", help = "chroot path to use for the ZooKeeper connections")
-  private static final Arg<String> CHROOT_PATH = Arg.create(null);
-
-  @CmdLine(name = "zk_session_timeout", help ="The ZooKeeper session timeout.")
-  private static final Arg<Amount<Integer, Time>> SESSION_TIMEOUT =
-      Arg.create(ZooKeeperUtils.DEFAULT_ZK_SESSION_TIMEOUT);
-
-  @CmdLine(name = "zk_digest_credentials",
-           help ="user:password to use when authenticating with ZooKeeper.")
-  private static final Arg<String> DIGEST_CREDENTIALS = Arg.create();
-
-  /**
-   * Creates a configuration from command line arguments.
-   *
-   * @return Configuration instance.
-   */
-  public static ClientConfig create() {
-    return new ClientConfig(
-        ZK_ENDPOINTS.get(),
-        Optional.fromNullable(CHROOT_PATH.get()),
-        IN_PROCESS.get(),
-        SESSION_TIMEOUT.get(),
-        DIGEST_CREDENTIALS.hasAppliedValue()
-            ? getCredentials(DIGEST_CREDENTIALS.get())
-            : Credentials.NONE
-    );
-  }
-
-  private static Credentials getCredentials(String userAndPass) {
-    List<String> parts = ImmutableList.copyOf(Splitter.on(":").split(userAndPass));
-    if (parts.size() != 2) {
-      throw new IllegalArgumentException(
-          "zk_digest_credentials must be formatted as user:pass");
-    }
-    return ZooKeeperClient.digestCredentials(parts.get(0), parts.get(1));
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/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 d97c001..170d37b 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
@@ -17,7 +17,6 @@ import java.io.IOException;
 
 import com.google.common.base.Preconditions;
 
-import org.apache.aurora.common.application.ShutdownRegistry.ShutdownRegistryImpl;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.testing.TearDownTestCase;
@@ -53,13 +52,8 @@ public abstract class BaseZooKeeperTest extends TearDownTestCase {
 
   @Before
   public final void setUp() throws Exception {
-    final ShutdownRegistryImpl shutdownRegistry = new ShutdownRegistryImpl();
-    addTearDown(new TearDown() {
-      @Override public void tearDown() {
-        shutdownRegistry.execute();
-      }
-    });
-    zkTestServer = new ZooKeeperTestServer(0, shutdownRegistry, defaultSessionTimeout);
+    zkTestServer = new ZooKeeperTestServer(defaultSessionTimeout);
+    addTearDown(zkTestServer::stop);
     zkTestServer.startNetwork();
   }
 
@@ -133,15 +127,6 @@ public abstract class BaseZooKeeperTest extends TearDownTestCase {
 
   /**
    * Returns a new authenticated zookeeper client connected to the in-process zookeeper server with
-   * a custom {@code sessionTimeout}.
-   */
-  protected final ZooKeeperClient createZkClient(Amount<Integer, Time> sessionTimeout,
-      ZooKeeperClient.Credentials credentials) {
-    return zkTestServer.createClient(sessionTimeout, credentials);
-  }
-
-  /**
-   * 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) {

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/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 afd2315..ac5d64a 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
@@ -17,23 +17,22 @@ 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 com.google.common.io.Files;
 
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient;
+import org.apache.commons.io.FileUtils;
 import org.apache.zookeeper.server.NIOServerCnxn;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer.BasicDataTreeBuilder;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 
-import org.apache.aurora.common.application.ShutdownRegistry;
-import org.apache.aurora.common.base.Command;
-import org.apache.aurora.common.base.ExceptionalCommand;
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.common.quantity.Time;
-
 /**
  * A helper class for starting in-process ZooKeeper server and clients.
  *
@@ -43,40 +42,30 @@ public class ZooKeeperTestServer {
 
   /**
    * The default session timeout for clients created by servers constructed with
-   * {@link #ZooKeeperTestServer(int, ShutdownRegistry)}.
+   * {@link #ZooKeeperTestServer()}.
    */
   public static final Amount<Integer, Time> DEFAULT_SESSION_TIMEOUT =
       Amount.of(100, Time.MILLISECONDS);
 
   protected final ZooKeeperServer zooKeeperServer;
-  private final ShutdownRegistry shutdownRegistry;
   private NIOServerCnxn.Factory connectionFactory;
   private int port;
   private final Amount<Integer, Time> defaultSessionTimeout;
+  private final LinkedList<Runnable> cleanupActions = Lists.newLinkedList();
 
   /**
-   * @param port the port to start the zoo keeper server on - {@code 0} picks an ephemeral port
-   * @param shutdownRegistry a registry that will be used to register client and server shutdown
-   *     commands.  It is up to the caller to execute the registered actions at an appropriate time.
    * @throws IOException if there was aproblem creating the server's database
    */
-  public ZooKeeperTestServer(int port, ShutdownRegistry shutdownRegistry) throws IOException {
-    this(port, shutdownRegistry, DEFAULT_SESSION_TIMEOUT);
+  public ZooKeeperTestServer() throws IOException {
+    this(DEFAULT_SESSION_TIMEOUT);
   }
 
   /**
-   * @param port the port to start the zoo keeper server on - {@code 0} picks an ephemeral port
-   * @param shutdownRegistry a registry that will be used to register client and server shutdown
-   *     commands.  It is up to the caller to execute the registered actions at an appropriate time.
    * @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(int port, ShutdownRegistry shutdownRegistry,
-      Amount<Integer, Time> defaultSessionTimeout) throws IOException {
-    Preconditions.checkArgument(0 <= port && port <= 0xFFFF);
-    this.port = port;
-    this.shutdownRegistry = Preconditions.checkNotNull(shutdownRegistry);
+  public ZooKeeperTestServer(Amount<Integer, Time> defaultSessionTimeout) throws IOException {
     this.defaultSessionTimeout = Preconditions.checkNotNull(defaultSessionTimeout);
 
     zooKeeperServer =
@@ -93,19 +82,23 @@ public class ZooKeeperTestServer {
   }
 
   /**
-   * Starts zookeeper up on the configured port.  If the configured port is the ephemeral port
-   * (@{code 0}), then the actual chosen port is returned.
+   * Starts zookeeper up on an ephemeral port.
    */
-  public final int startNetwork() throws IOException, InterruptedException {
+  public void startNetwork() throws IOException, InterruptedException {
     connectionFactory = new NIOServerCnxn.Factory(new InetSocketAddress(port));
     connectionFactory.startup(zooKeeperServer);
-    shutdownRegistry.addAction(new Command() {
-      @Override public void execute() {
-        shutdownNetwork();
-      }
-    });
+    cleanupActions.addFirst((this::shutdownNetwork));
     port = zooKeeperServer.getClientPort();
-    return port;
+  }
+
+  /**
+   * Stops the zookeeper server.
+   */
+  public void stop() {
+    for (Runnable cleanup : cleanupActions) {
+      cleanup.run();
+    }
+    cleanupActions.clear();
   }
 
   /**
@@ -195,13 +188,10 @@ public class ZooKeeperTestServer {
    */
   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)));
-    shutdownRegistry.addAction(new ExceptionalCommand<InterruptedException>() {
-      @Override public void execute() {
-        client.close();
-      }
-    });
+    cleanupActions.addFirst(client::close);
     return client;
   }
 
@@ -211,9 +201,14 @@ public class ZooKeeperTestServer {
 
   private File createTempDir() {
     final File tempDir = Files.createTempDir();
-    shutdownRegistry.addAction(new ExceptionalCommand<IOException>() {
-      @Override public void execute() throws IOException {
-        org.apache.commons.io.FileUtils.deleteDirectory(tempDir);
+    cleanupActions.addFirst(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          FileUtils.deleteDirectory(tempDir);
+        } catch (IOException e) {
+          // No-op.
+        }
       }
     });
     return tempDir;

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/config/legacy_untested_classes.txt
----------------------------------------------------------------------
diff --git a/config/legacy_untested_classes.txt b/config/legacy_untested_classes.txt
index 346b947..07fd5f1 100644
--- a/config/legacy_untested_classes.txt
+++ b/config/legacy_untested_classes.txt
@@ -60,3 +60,7 @@ org/apache/aurora/scheduler/storage/mem/Util
 org/apache/aurora/scheduler/storage/mem/Util$1
 org/apache/aurora/scheduler/testing/FakeStatsProvider$3
 org/apache/aurora/scheduler/updater/UpdateConfigurationException
+org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule$LocalClientProvider
+org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule$TestServerService
+org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule$1
+org/apache/aurora/scheduler/zookeeper/guice/client/flagged/FlaggedClientConfig

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/AppModule.java b/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
index bf8ab31..78071ed 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
@@ -34,7 +34,6 @@ import org.apache.aurora.common.zookeeper.SingletonService;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient.Credentials;
 import org.apache.aurora.common.zookeeper.ZooKeeperUtils;
-import org.apache.aurora.common.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
 import org.apache.aurora.gen.ServerInfo;
 import org.apache.aurora.scheduler.SchedulerModule;
 import org.apache.aurora.scheduler.SchedulerServicesModule;
@@ -55,6 +54,7 @@ import org.apache.aurora.scheduler.state.StateModule;
 import org.apache.aurora.scheduler.stats.AsyncStatsModule;
 import org.apache.aurora.scheduler.storage.entities.IServerInfo;
 import org.apache.aurora.scheduler.updater.UpdaterModule;
+import org.apache.aurora.scheduler.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
 import org.apache.mesos.Scheduler;
 import org.apache.zookeeper.data.ACL;
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java b/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
index 89399f1..11f04da 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
@@ -46,9 +46,6 @@ import org.apache.aurora.common.quantity.Data;
 import org.apache.aurora.common.zookeeper.Group;
 import org.apache.aurora.common.zookeeper.SingletonService;
 import org.apache.aurora.common.zookeeper.SingletonService.LeadershipListener;
-import org.apache.aurora.common.zookeeper.guice.client.ZooKeeperClientModule;
-import org.apache.aurora.common.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
-import org.apache.aurora.common.zookeeper.guice.client.flagged.FlaggedClientConfig;
 import org.apache.aurora.gen.Volume;
 import org.apache.aurora.scheduler.AppStartup;
 import org.apache.aurora.scheduler.ResourceSlot;
@@ -65,6 +62,9 @@ import org.apache.aurora.scheduler.storage.backup.BackupModule;
 import org.apache.aurora.scheduler.storage.db.DbModule;
 import org.apache.aurora.scheduler.storage.log.LogStorageModule;
 import org.apache.aurora.scheduler.storage.log.SnapshotStoreImpl;
+import org.apache.aurora.scheduler.zookeeper.guice.client.ZooKeeperClientModule;
+import org.apache.aurora.scheduler.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
+import org.apache.aurora.scheduler.zookeeper.guice.client.flagged.FlaggedClientConfig;
 
 import static org.apache.aurora.common.logging.RootLogConfig.Configuration;
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/src/main/java/org/apache/aurora/scheduler/log/mesos/MesosLogStreamModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/log/mesos/MesosLogStreamModule.java b/src/main/java/org/apache/aurora/scheduler/log/mesos/MesosLogStreamModule.java
index e47dbaa..377dbfd 100644
--- a/src/main/java/org/apache/aurora/scheduler/log/mesos/MesosLogStreamModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/log/mesos/MesosLogStreamModule.java
@@ -33,10 +33,10 @@ import org.apache.aurora.common.args.CmdLine;
 import org.apache.aurora.common.net.InetSocketAddressHelper;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Time;
-import org.apache.aurora.common.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
 import org.apache.aurora.gen.storage.LogEntry;
 import org.apache.aurora.scheduler.log.mesos.LogInterface.ReaderInterface;
 import org.apache.aurora.scheduler.log.mesos.LogInterface.WriterInterface;
+import org.apache.aurora.scheduler.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
 import org.apache.mesos.Log;
 import org.apache.zookeeper.common.PathUtils;
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/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
new file mode 100644
index 0000000..273883f
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/ZooKeeperClientModule.java
@@ -0,0 +1,200 @@
+/**
+ * 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.scheduler.zookeeper.guice.client;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Throwables;
+import com.google.common.util.concurrent.AbstractIdleService;
+import com.google.inject.AbstractModule;
+import com.google.inject.Inject;
+import com.google.inject.Key;
+import com.google.inject.PrivateModule;
+import com.google.inject.Provider;
+import com.google.inject.Singleton;
+
+import org.apache.aurora.common.application.ShutdownRegistry;
+import org.apache.aurora.common.inject.Bindings.KeyFactory;
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.common.quantity.Time;
+import org.apache.aurora.common.zookeeper.ZooKeeperClient;
+import org.apache.aurora.common.zookeeper.ZooKeeperClient.Credentials;
+import org.apache.aurora.common.zookeeper.ZooKeeperUtils;
+import org.apache.aurora.common.zookeeper.testing.ZooKeeperTestServer;
+import org.apache.aurora.scheduler.SchedulerServicesModule;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A guice binding module that configures and binds a {@link ZooKeeperClient} instance.
+ */
+public class ZooKeeperClientModule extends AbstractModule {
+  private final KeyFactory keyFactory;
+  private final ClientConfig config;
+
+  /**
+   * Creates a new ZK client module from the provided configuration.
+   *
+   * @param config Configuration parameters for the client.
+   */
+  public ZooKeeperClientModule(ClientConfig config) {
+    this(KeyFactory.PLAIN, config);
+  }
+
+  /**
+   * Creates a new ZK client module from the provided configuration, using a key factory to
+   * qualify any bindings.
+   *
+   * @param keyFactory Factory to use when creating any exposed bindings.
+   * @param config Configuration parameters for the client.
+   */
+  public ZooKeeperClientModule(KeyFactory keyFactory, ClientConfig config) {
+    this.keyFactory = checkNotNull(keyFactory);
+    this.config = checkNotNull(config);
+  }
+
+  @Override
+  protected void configure() {
+    Key<ZooKeeperClient> clientKey = keyFactory.create(ZooKeeperClient.class);
+    if (config.inProcess) {
+      install(new PrivateModule() {
+        @Override
+        protected void configure() {
+          requireBinding(ShutdownRegistry.class);
+          // Bound privately to give the local provider access to configuration settings.
+          bind(ClientConfig.class).toInstance(config);
+          bind(clientKey).toProvider(LocalClientProvider.class).in(Singleton.class);
+          expose(clientKey);
+        }
+      });
+      bind(ZooKeeperTestServer.class).in(Singleton.class);
+      SchedulerServicesModule.addAppStartupServiceBinding(binder()).to(TestServerService.class);
+    } else {
+      bind(clientKey).toInstance(new ZooKeeperClient(
+          config.sessionTimeout,
+          config.credentials,
+          config.chrootPath,
+          config.servers));
+    }
+  }
+
+  /**
+   * A service to wrap ZooKeeperTestServer.  ZooKeeperTestServer is not a service itself because
+   * some tests depend on stop/start routines that do not no-op, like startAsync and stopAsync may.
+   */
+  private static class TestServerService extends AbstractIdleService {
+    private final ZooKeeperTestServer testServer;
+
+    @Inject
+    TestServerService(ZooKeeperTestServer testServer) {
+      this.testServer = checkNotNull(testServer);
+    }
+
+    @Override
+    protected void startUp() {
+      // We actually start the test server on-demand rather than with the normal lifecycle.
+      // This is because a ZooKeeperClient binding is needed before scheduler services are started.
+    }
+
+    @Override
+    protected void shutDown() {
+      testServer.stop();
+    }
+  }
+
+  private static class LocalClientProvider implements Provider<ZooKeeperClient> {
+    private final ClientConfig config;
+    private final ZooKeeperTestServer testServer;
+
+    @Inject
+    LocalClientProvider(ClientConfig config, ZooKeeperTestServer testServer) {
+      this.config = checkNotNull(config);
+      this.testServer = checkNotNull(testServer);
+    }
+
+    @Override
+    public ZooKeeperClient get() {
+      try {
+        testServer.startNetwork();
+      } catch (IOException | InterruptedException e) {
+        throw Throwables.propagate(e);
+      }
+      return testServer.createClient(config.sessionTimeout, config.credentials);
+    }
+  }
+
+  /**
+   * Composite type that contains configuration parameters used when creating a client.
+   * <p>
+   * Instances of this class are immutable, but builder-style chained calls are supported.
+   */
+  public static class ClientConfig {
+    public final Iterable<InetSocketAddress> servers;
+    public final boolean inProcess;
+    public final Amount<Integer, Time> sessionTimeout;
+    public final Optional<String> chrootPath;
+    public final Credentials credentials;
+
+    /**
+     * Creates a new client configuration.
+     *
+     * @param servers ZooKeeper server addresses.
+     * @param inProcess Whether to run and create clients for an in-process ZooKeeper server.
+     * @param chrootPath an optional chroot path
+     * @param sessionTimeout Timeout duration for established sessions.
+     * @param credentials ZooKeeper authentication credentials.
+     */
+    public ClientConfig(
+        Iterable<InetSocketAddress> servers,
+        Optional<String> chrootPath,
+        boolean inProcess,
+        Amount<Integer, Time> sessionTimeout,
+        Credentials credentials) {
+
+      this.servers = servers;
+      this.chrootPath = chrootPath;
+      this.inProcess = inProcess;
+      this.sessionTimeout = sessionTimeout;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Creates a new client configuration with defaults for the session timeout and credentials.
+     *
+     * @param servers ZooKeeper server addresses.
+     * @return A new configuration.
+     */
+    public static ClientConfig create(Iterable<InetSocketAddress> servers) {
+      return new ClientConfig(
+          servers,
+          Optional.<String> absent(),
+          false,
+          ZooKeeperUtils.DEFAULT_ZK_SESSION_TIMEOUT,
+          Credentials.NONE);
+    }
+
+    /**
+     * Creates a new configuration identical to this configuration, but with the provided
+     * credentials.
+     *
+     * @param newCredentials ZooKeeper authentication credentials.
+     * @return A modified clone of this configuration.
+     */
+    public ClientConfig withCredentials(Credentials newCredentials) {
+      return new ClientConfig(servers, chrootPath, inProcess, sessionTimeout, newCredentials);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/flagged/FlaggedClientConfig.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/flagged/FlaggedClientConfig.java b/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/flagged/FlaggedClientConfig.java
new file mode 100644
index 0000000..5b59d55
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/zookeeper/guice/client/flagged/FlaggedClientConfig.java
@@ -0,0 +1,86 @@
+/**
+ * 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.scheduler.zookeeper.guice.client.flagged;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.aurora.common.args.Arg;
+import org.apache.aurora.common.args.CmdLine;
+import org.apache.aurora.common.args.constraints.NotEmpty;
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.common.quantity.Time;
+import org.apache.aurora.common.zookeeper.ZooKeeperClient;
+import org.apache.aurora.common.zookeeper.ZooKeeperClient.Credentials;
+import org.apache.aurora.common.zookeeper.ZooKeeperUtils;
+import org.apache.aurora.scheduler.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
+
+/**
+ * A factory that creates a {@link ClientConfig} instance based on command line argument values.
+ */
+public final class FlaggedClientConfig {
+  @CmdLine(name = "zk_in_proc",
+      help = "Launches an embedded zookeeper server for local testing causing -zk_endpoints "
+          + "to be ignored if specified.")
+  private static final Arg<Boolean> IN_PROCESS = Arg.create(false);
+
+  @NotEmpty
+  @CmdLine(name = "zk_endpoints", help = "Endpoint specification for the ZooKeeper servers.")
+  private static final Arg<List<InetSocketAddress>> ZK_ENDPOINTS = Arg.create();
+
+  @CmdLine(name = "zk_chroot_path", help = "chroot path to use for the ZooKeeper connections")
+  private static final Arg<String> CHROOT_PATH = Arg.create(null);
+
+  @CmdLine(name = "zk_session_timeout", help = "The ZooKeeper session timeout.")
+  private static final Arg<Amount<Integer, Time>> SESSION_TIMEOUT =
+      Arg.create(ZooKeeperUtils.DEFAULT_ZK_SESSION_TIMEOUT);
+
+  @CmdLine(name = "zk_digest_credentials",
+           help = "user:password to use when authenticating with ZooKeeper.")
+  private static final Arg<String> DIGEST_CREDENTIALS = Arg.create();
+
+  private FlaggedClientConfig() {
+    // Utility class.
+  }
+
+  /**
+   * Creates a configuration from command line arguments.
+   *
+   * @return Configuration instance.
+   */
+  public static ClientConfig create() {
+    return new ClientConfig(
+        ZK_ENDPOINTS.get(),
+        Optional.fromNullable(CHROOT_PATH.get()),
+        IN_PROCESS.get(),
+        SESSION_TIMEOUT.get(),
+        DIGEST_CREDENTIALS.hasAppliedValue()
+            ? getCredentials(DIGEST_CREDENTIALS.get())
+            : Credentials.NONE
+    );
+  }
+
+  private static Credentials getCredentials(String userAndPass) {
+    List<String> parts = ImmutableList.copyOf(Splitter.on(":").split(userAndPass));
+    if (parts.size() != 2) {
+      throw new IllegalArgumentException(
+          "zk_digest_credentials must be formatted as user:pass");
+    }
+    return ZooKeeperClient.digestCredentials(parts.get(0), parts.get(1));
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5dccf92f/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 7f141df..4941128 100644
--- a/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
@@ -55,8 +55,6 @@ import org.apache.aurora.common.thrift.ServiceInstance;
 import org.apache.aurora.common.zookeeper.ServerSet;
 import org.apache.aurora.common.zookeeper.ServerSetImpl;
 import org.apache.aurora.common.zookeeper.ZooKeeperClient;
-import org.apache.aurora.common.zookeeper.guice.client.ZooKeeperClientModule;
-import org.apache.aurora.common.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
 import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
 import org.apache.aurora.gen.AssignedTask;
 import org.apache.aurora.gen.Identity;
@@ -89,6 +87,8 @@ import org.apache.aurora.scheduler.storage.log.LogStorageModule;
 import org.apache.aurora.scheduler.storage.log.SnapshotStoreImpl;
 import org.apache.aurora.scheduler.storage.log.testing.LogOpMatcher;
 import org.apache.aurora.scheduler.storage.log.testing.LogOpMatcher.StreamMatcher;
+import org.apache.aurora.scheduler.zookeeper.guice.client.ZooKeeperClientModule;
+import org.apache.aurora.scheduler.zookeeper.guice.client.ZooKeeperClientModule.ClientConfig;
 import org.apache.mesos.Protos.FrameworkID;
 import org.apache.mesos.Protos.MasterInfo;
 import org.apache.mesos.Protos.Status;