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/15 03:31:09 UTC

aurora git commit: Refactor SchedulerMain to absorb AppLauncher.

Repository: aurora
Updated Branches:
  refs/heads/master e658a8a0b -> 44e472642


Refactor SchedulerMain to absorb AppLauncher.

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


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

Branch: refs/heads/master
Commit: 44e47264261fc01def325eccd34bfa0d5079e479
Parents: e658a8a
Author: Bill Farner <wf...@apache.org>
Authored: Mon Sep 14 18:31:05 2015 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Mon Sep 14 18:31:05 2015 -0700

----------------------------------------------------------------------
 .../aurora/common/application/AppLauncher.java  | 101 ---------
 .../aurora/common/application/Application.java  |  29 ---
 .../aurora/common/application/Lifecycle.java    |   6 +-
 .../application/modules/AppLauncherModule.java  |  48 ----
 config/checkstyle/checkstyle.xml                |   6 +-
 config/checkstyle/suppressions.xml              |  24 ++
 config/legacy_untested_classes.txt              |   1 +
 .../apache/aurora/scheduler/app/AppModule.java  |  84 -------
 .../apache/aurora/scheduler/app/Modules.java    |  98 --------
 .../aurora/scheduler/app/MoreModules.java       |  98 ++++++++
 .../aurora/scheduler/app/SchedulerMain.java     | 226 +++++++++++--------
 .../scheduler/app/ServiceDiscoveryModule.java   |  91 ++++++++
 .../http/api/security/HttpSecurityModule.java   |   4 +-
 .../http/api/security/ModuleParser.java         |   4 +-
 .../scheduler/thrift/auth/ThriftAuthModule.java |   4 +-
 .../scheduler/SchedulerLifecycleTest.java       |   8 +-
 .../aurora/scheduler/app/ModulesTest.java       |  39 ----
 .../aurora/scheduler/app/MoreModulesTest.java   |  39 ++++
 .../aurora/scheduler/app/SchedulerIT.java       |  24 +-
 .../scheduler/app/local/LocalSchedulerMain.java |  64 +++---
 .../aurora/scheduler/http/QuitCallbackTest.java |   6 +-
 .../scheduler/log/mesos/MesosLogTest.java       |   2 +-
 .../scheduler/mesos/MesosSchedulerImplTest.java |   5 +-
 .../aurora/scheduler/sla/SlaModuleTest.java     |   2 -
 24 files changed, 445 insertions(+), 568 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/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
deleted file mode 100644
index 6496870..0000000
--- a/commons/src/main/java/org/apache/aurora/common/application/AppLauncher.java
+++ /dev/null
@@ -1,101 +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.application;
-
-import java.util.Arrays;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import com.google.common.base.Preconditions;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-
-import org.apache.aurora.common.args.Arg;
-import org.apache.aurora.common.args.ArgFilters;
-import org.apache.aurora.common.args.ArgScanner;
-import org.apache.aurora.common.args.ArgScanner.ArgScanException;
-import org.apache.aurora.common.args.CmdLine;
-
-/**
- * An application launcher that sets up a framework for pluggable binding modules.  This class
- * should be called directly as the main class, with a command line argument {@code -app_class}
- * which is the canonical class name of the application to execute.
- *
- * If your application uses command line arguments all {@link Arg} fields annotated with
- * {@link CmdLine} will be discovered and command line arguments will be validated against this set,
- * parsed and applied.
- */
-public final class AppLauncher {
-
-  private static final Logger LOG = Logger.getLogger(AppLauncher.class.getName());
-
-  private AppLauncher() {
-    // This should not be invoked directly.
-  }
-
-  private void run(Application application) {
-    Lifecycle lifecycle = null;
-    try {
-      Injector injector = Guice.createInjector(application.getModules());
-      lifecycle = injector.getInstance(Lifecycle.class);
-      injector.injectMembers(application);
-      try {
-        application.run();
-      } finally {
-        LOG.info("Application run() exited.");
-      }
-    } finally {
-      if (lifecycle != null) {
-        lifecycle.shutdown();
-      }
-    }
-  }
-
-  /**
-   * Used to launch an application with a restricted set of {@literal @CmdLine} {@link Arg}s
-   * considered for parsing.  This is useful if the classpath includes annotated fields you do not
-   * wish arguments to be parsed for.
-   *
-   * @param appClass The application class to instantiate and launch.
-   * @param args The command line arguments to parse.
-   * @see ArgFilters
-   */
-  public static void launch(Class<? extends Application> appClass, String... args) {
-    Preconditions.checkNotNull(appClass);
-    Preconditions.checkNotNull(args);
-
-    try {
-      if (!new ArgScanner().parse(Arrays.asList(args))) {
-        System.exit(0);
-      }
-    } catch (ArgScanException e) {
-      exit("Failed to scan arguments", e);
-    } catch (IllegalArgumentException e) {
-      exit("Failed to apply arguments", e);
-    }
-
-    try {
-      new AppLauncher().run(appClass.newInstance());
-    } catch (InstantiationException e) {
-      throw new IllegalStateException(e);
-    } catch (IllegalAccessException e) {
-      throw new IllegalStateException(e);
-    }
-  }
-
-  private static void exit(String message, Exception error) {
-    LOG.log(Level.SEVERE, message + "\n" + error, error);
-    System.exit(1);
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/commons/src/main/java/org/apache/aurora/common/application/Application.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/application/Application.java b/commons/src/main/java/org/apache/aurora/common/application/Application.java
deleted file mode 100644
index ebe01e8..0000000
--- a/commons/src/main/java/org/apache/aurora/common/application/Application.java
+++ /dev/null
@@ -1,29 +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.application;
-
-import com.google.inject.Module;
-
-/**
- * An application that supports a limited lifecycle and optional binding of guice modules.
- */
-public interface Application extends Runnable {
-
-  /**
-   * Returns binding modules for the application.
-   *
-   * @return Application binding modules.
-   */
-  Iterable<? extends Module> getModules();
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/commons/src/main/java/org/apache/aurora/common/application/Lifecycle.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/application/Lifecycle.java b/commons/src/main/java/org/apache/aurora/common/application/Lifecycle.java
index 5d22a2a..a71a51d 100644
--- a/commons/src/main/java/org/apache/aurora/common/application/Lifecycle.java
+++ b/commons/src/main/java/org/apache/aurora/common/application/Lifecycle.java
@@ -13,7 +13,6 @@
  */
 package org.apache.aurora.common.application;
 
-import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.logging.Logger;
 
 import com.google.inject.Inject;
@@ -38,11 +37,8 @@ public class Lifecycle {
   private final Command shutdownRegistry;
 
   @Inject
-  public Lifecycle(@ShutdownStage Command shutdownRegistry,
-      UncaughtExceptionHandler exceptionHandler) {
-
+  public Lifecycle(@ShutdownStage Command shutdownRegistry) {
     this.shutdownRegistry = shutdownRegistry;
-    Thread.setDefaultUncaughtExceptionHandler(exceptionHandler);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/commons/src/main/java/org/apache/aurora/common/application/modules/AppLauncherModule.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/application/modules/AppLauncherModule.java b/commons/src/main/java/org/apache/aurora/common/application/modules/AppLauncherModule.java
deleted file mode 100644
index 1c08f7e..0000000
--- a/commons/src/main/java/org/apache/aurora/common/application/modules/AppLauncherModule.java
+++ /dev/null
@@ -1,48 +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.application.modules;
-
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import com.google.inject.AbstractModule;
-
-import org.apache.aurora.common.application.AppLauncher;
-import org.apache.aurora.common.stats.Stats;
-
-/**
- * Binding module for the bare minimum requirements for the
- * {@link AppLauncher}.
- *
- * @author William Farner
- */
-public class AppLauncherModule extends AbstractModule {
-
-  private static final Logger LOG = Logger.getLogger(AppLauncherModule.class.getName());
-  private static final AtomicLong UNCAUGHT_EXCEPTIONS = Stats.exportLong("uncaught_exceptions");
-
-  @Override
-  protected void configure() {
-    bind(UncaughtExceptionHandler.class).to(LoggingExceptionHandler.class);
-  }
-
-  public static class LoggingExceptionHandler implements UncaughtExceptionHandler {
-    @Override public void uncaughtException(Thread t, Throwable e) {
-      UNCAUGHT_EXCEPTIONS.incrementAndGet();
-      LOG.log(Level.SEVERE, "Uncaught exception from " + t + ":" + e, e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/config/checkstyle/checkstyle.xml
----------------------------------------------------------------------
diff --git a/config/checkstyle/checkstyle.xml b/config/checkstyle/checkstyle.xml
index 9c291f5..2074beb 100644
--- a/config/checkstyle/checkstyle.xml
+++ b/config/checkstyle/checkstyle.xml
@@ -266,7 +266,7 @@ limitations under the License.
     <module name="ArrayTypeStyle"/>
     <module name="OuterTypeFilename"/>
 
-    <!-- Regexp checks -->
+    <!-- Discourage use of System.exit() -->
     <module name="RegexpSinglelineJava">
       <property name="format" value="System.exit"/>
       <property name="ignoreComments" value="true"/>
@@ -274,4 +274,8 @@ limitations under the License.
         value="Don't System.exit(), throw a RuntimeException()" />
     </module>
   </module>
+
+  <module name="SuppressionFilter">
+    <property name="file" value="config/checkstyle/suppressions.xml"/>
+  </module>
 </module>

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/config/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml
new file mode 100644
index 0000000..50a53e0
--- /dev/null
+++ b/config/checkstyle/suppressions.xml
@@ -0,0 +1,24 @@
+<?xml version="1.0"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this work except in compliance with the License.
+You may obtain a copy of the License in the LICENSE file, or 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.
+-->
+
+<!DOCTYPE suppressions PUBLIC
+    "-//Puppy Crawl//DTD Suppressions 1.1//EN"
+    "http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
+
+<suppressions>
+  <!-- Allow use of System.exit() in main. -->
+  <suppress files="org/apache/aurora/scheduler/app/SchedulerMain.java"
+            checks="RegexpSinglelineJava"/>
+</suppressions>

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/config/legacy_untested_classes.txt
----------------------------------------------------------------------
diff --git a/config/legacy_untested_classes.txt b/config/legacy_untested_classes.txt
index 07fd5f1..7b891b2 100644
--- a/config/legacy_untested_classes.txt
+++ b/config/legacy_untested_classes.txt
@@ -1,6 +1,7 @@
 org/apache/aurora/Protobufs$1
 org/apache/aurora/auth/UnsecureAuthModule$UnsecureCapabilityValidator$1
 org/apache/aurora/auth/UnsecureAuthModule$UnsecureCapabilityValidator$2
+org/apache/aurora/scheduler/app/SchedulerMain$1
 org/apache/aurora/scheduler/app/SchedulerMain$2
 org/apache/aurora/scheduler/app/SchedulerMain$3
 org/apache/aurora/scheduler/app/SchedulerMain$4

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/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 78071ed..967e10d 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
@@ -13,28 +13,15 @@
  */
 package org.apache.aurora.scheduler.app;
 
-import java.util.List;
-import java.util.logging.Logger;
-
 import javax.inject.Singleton;
 
 import com.google.inject.AbstractModule;
-import com.google.inject.Provides;
 
 import org.apache.aurora.GuiceUtils;
 import org.apache.aurora.common.inject.TimedInterceptor;
-import org.apache.aurora.common.net.pool.DynamicHostSet;
 import org.apache.aurora.common.stats.Stats;
 import org.apache.aurora.common.stats.StatsProvider;
-import org.apache.aurora.common.thrift.ServiceInstance;
 import org.apache.aurora.common.util.Clock;
-import org.apache.aurora.common.zookeeper.ServerSet;
-import org.apache.aurora.common.zookeeper.ServerSetImpl;
-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.gen.ServerInfo;
 import org.apache.aurora.scheduler.SchedulerModule;
 import org.apache.aurora.scheduler.SchedulerServicesModule;
 import org.apache.aurora.scheduler.async.AsyncModule;
@@ -52,40 +39,13 @@ import org.apache.aurora.scheduler.scheduling.SchedulingModule;
 import org.apache.aurora.scheduler.sla.SlaModule;
 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;
-
-import static java.util.Objects.requireNonNull;
-
-import static org.apache.aurora.common.base.MorePreconditions.checkNotBlank;
-import static org.apache.aurora.gen.apiConstants.THRIFT_API_VERSION;
 
 /**
  * Binding module for the aurora scheduler application.
  */
 public class AppModule extends AbstractModule {
-  private static final Logger LOG = Logger.getLogger(AppModule.class.getName());
-
-  private final String clusterName;
-  private final String serverSetPath;
-  private final ClientConfig zkClientConfig;
-  private final String statsUrlPrefix;
-
-  AppModule(
-      String clusterName,
-      String serverSetPath,
-      ClientConfig zkClientConfig,
-      String statsUrlPrefix) {
-
-    this.clusterName = checkNotBlank(clusterName);
-    this.serverSetPath = checkNotBlank(serverSetPath);
-    this.zkClientConfig = requireNonNull(zkClientConfig);
-    this.statsUrlPrefix = statsUrlPrefix;
-  }
-
   @Override
   protected void configure() {
     // Enable intercepted method timings and context classloader repair.
@@ -94,14 +54,6 @@ public class AppModule extends AbstractModule {
     GuiceUtils.bindExceptionTrap(binder(), Scheduler.class);
 
     bind(Clock.class).toInstance(Clock.SYSTEM_CLOCK);
-
-    bind(IServerInfo.class).toInstance(
-        IServerInfo.build(
-            new ServerInfo()
-                .setClusterName(clusterName)
-                .setThriftAPIVersion(THRIFT_API_VERSION)
-                .setStatsUrlPrefix(statsUrlPrefix)));
-
     install(new PubsubEventModule());
     // Filter layering: notifier filter -> base impl
     PubsubEventModule.bindSchedulingFilterDelegate(binder()).to(SchedulingFilterImpl.class);
@@ -125,40 +77,4 @@ public class AppModule extends AbstractModule {
     install(new UpdaterModule());
     bind(StatsProvider.class).toInstance(Stats.STATS_PROVIDER);
   }
-
-  @Provides
-  @Singleton
-  List<ACL> provideAcls() {
-    if (zkClientConfig.credentials == Credentials.NONE) {
-      LOG.warning("Running without ZooKeeper digest credentials. ZooKeeper ACLs are disabled.");
-      return ZooKeeperUtils.OPEN_ACL_UNSAFE;
-    } else {
-      return ZooKeeperUtils.EVERYONE_READ_CREATOR_ALL;
-    }
-  }
-
-  @Provides
-  @Singleton
-  ServerSet provideServerSet(ZooKeeperClient client, List<ACL> zooKeeperAcls) {
-    return new ServerSetImpl(client, zooKeeperAcls, serverSetPath);
-  }
-
-  @Provides
-  @Singleton
-  DynamicHostSet<ServiceInstance> provideSchedulerHostSet(ServerSet serverSet) {
-    // Used for a type re-binding of the serverset.
-    return serverSet;
-  }
-
-  @Provides
-  @Singleton
-  SingletonService provideSingletonService(
-      ZooKeeperClient client,
-      ServerSet serverSet,
-      List<ACL> zookeeperAcls) {
-
-    return new SingletonService(
-        serverSet,
-        SingletonService.createSingletonCandidate(client, serverSetPath, zookeeperAcls));
-  }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/main/java/org/apache/aurora/scheduler/app/Modules.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/Modules.java b/src/main/java/org/apache/aurora/scheduler/app/Modules.java
deleted file mode 100644
index cad7c38..0000000
--- a/src/main/java/org/apache/aurora/scheduler/app/Modules.java
+++ /dev/null
@@ -1,98 +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.scheduler.app;
-
-import com.google.inject.AbstractModule;
-import com.google.inject.Module;
-import com.google.inject.PrivateModule;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * A utility class for managing guice modules.
- */
-public final class Modules {
-
-  private Modules() {
-    // Utility class
-  }
-
-  private static Module instantiateModule(final Class<? extends Module> moduleClass) {
-    try {
-      return moduleClass.newInstance();
-    } catch (InstantiationException e) {
-      throw new IllegalArgumentException(
-          String.format(
-              "Failed to instantiate module %s. Are you sure it has a no-arg constructor?",
-              moduleClass.getName()),
-          e);
-    } catch (IllegalAccessException e) {
-      throw new IllegalArgumentException(
-          String.format(
-              "Failed to instantiate module %s. Are you sure it's public?",
-              moduleClass.getName()),
-          e);
-    }
-  }
-
-  /**
-   * Defensively wrap a module in a PrivateModule that only exposes requested keys to ensure that
-   * we don't depend on surprise extra bindings across different implementations.
-   *
-   * @param module Module to wrap.
-   * @param exposedClasses Keys to expose.
-   * @return A private module that exposes the requested keys.
-   */
-  public static Module wrapInPrivateModule(
-      final Module module,
-      final Iterable<Class<?>> exposedClasses) {
-
-    requireNonNull(module);
-    requireNonNull(exposedClasses);
-
-    return new PrivateModule() {
-      @Override
-      protected void configure() {
-        install(module);
-        for (Class<?> klass : exposedClasses) {
-          expose(klass);
-        }
-      }
-    };
-  }
-
-  static Module getModule(Class<? extends Module> moduleClass) {
-    return instantiateModule(moduleClass);
-  }
-
-  /**
-   * Creates a module that will lazily instantiate and install another module.
-   * <p/>
-   * This serves as an indirection between module procurement and installation, which is necessary
-   * in cases where a module is referenced within a static initializer.  In this scenario, a module
-   * must not be instantiated if it reads command line arguments, as the args system has not yet
-   * had a chance to populate them.
-   *
-   * @param moduleClass Module to install.
-   * @return An installer that will install {@code moduleClass}.
-   */
-  public static Module lazilyInstantiated(final Class<? extends Module> moduleClass) {
-    return new AbstractModule() {
-      @Override
-      protected void configure() {
-        install(getModule(moduleClass));
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java b/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java
new file mode 100644
index 0000000..b79ccc8
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/app/MoreModules.java
@@ -0,0 +1,98 @@
+/**
+ * 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.app;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Module;
+import com.google.inject.PrivateModule;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A utility class for managing guice modules.
+ */
+public final class MoreModules {
+
+  private MoreModules() {
+    // Utility class
+  }
+
+  private static Module instantiateModule(final Class<? extends Module> moduleClass) {
+    try {
+      return moduleClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Failed to instantiate module %s. Are you sure it has a no-arg constructor?",
+              moduleClass.getName()),
+          e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Failed to instantiate module %s. Are you sure it's public?",
+              moduleClass.getName()),
+          e);
+    }
+  }
+
+  /**
+   * Defensively wrap a module in a PrivateModule that only exposes requested keys to ensure that
+   * we don't depend on surprise extra bindings across different implementations.
+   *
+   * @param module Module to wrap.
+   * @param exposedClasses Keys to expose.
+   * @return A private module that exposes the requested keys.
+   */
+  public static Module wrapInPrivateModule(
+      final Module module,
+      final Iterable<Class<?>> exposedClasses) {
+
+    requireNonNull(module);
+    requireNonNull(exposedClasses);
+
+    return new PrivateModule() {
+      @Override
+      protected void configure() {
+        install(module);
+        for (Class<?> klass : exposedClasses) {
+          expose(klass);
+        }
+      }
+    };
+  }
+
+  static Module getModule(Class<? extends Module> moduleClass) {
+    return instantiateModule(moduleClass);
+  }
+
+  /**
+   * Creates a module that will lazily instantiate and install another module.
+   * <p/>
+   * This serves as an indirection between module procurement and installation, which is necessary
+   * in cases where a module is referenced within a static initializer.  In this scenario, a module
+   * must not be instantiated if it reads command line arguments, as the args system has not yet
+   * had a chance to populate them.
+   *
+   * @param moduleClass Module to install.
+   * @return An installer that will install {@code moduleClass}.
+   */
+  public static Module lazilyInstantiated(final Class<? extends Module> moduleClass) {
+    return new AbstractModule() {
+      @Override
+      protected void configure() {
+        install(getModule(moduleClass));
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/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 11f04da..99b8744 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
@@ -13,10 +13,14 @@
  */
 package org.apache.aurora.scheduler.app;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.net.InetSocketAddress;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import javax.inject.Inject;
@@ -24,18 +28,20 @@ import javax.inject.Inject;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableList.Builder;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
 import com.google.common.net.HostAndPort;
 import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
 import com.google.inject.Module;
+import com.google.inject.util.Modules;
 
 import org.apache.aurora.GuavaUtils.ServiceManagerIface;
-import org.apache.aurora.common.application.AppLauncher;
-import org.apache.aurora.common.application.Application;
 import org.apache.aurora.common.application.Lifecycle;
-import org.apache.aurora.common.application.modules.AppLauncherModule;
 import org.apache.aurora.common.args.Arg;
+import org.apache.aurora.common.args.ArgScanner;
+import org.apache.aurora.common.args.ArgScanner.ArgScanException;
 import org.apache.aurora.common.args.CmdLine;
 import org.apache.aurora.common.args.constraints.NotEmpty;
 import org.apache.aurora.common.args.constraints.NotNull;
@@ -43,9 +49,11 @@ import org.apache.aurora.common.inject.Bindings;
 import org.apache.aurora.common.logging.RootLogConfig;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Data;
+import org.apache.aurora.common.stats.Stats;
 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.gen.ServerInfo;
 import org.apache.aurora.gen.Volume;
 import org.apache.aurora.scheduler.AppStartup;
 import org.apache.aurora.scheduler.ResourceSlot;
@@ -60,6 +68,7 @@ import org.apache.aurora.scheduler.stats.StatsModule;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.backup.BackupModule;
 import org.apache.aurora.scheduler.storage.db.DbModule;
+import org.apache.aurora.scheduler.storage.entities.IServerInfo;
 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;
@@ -67,11 +76,12 @@ import org.apache.aurora.scheduler.zookeeper.guice.client.ZooKeeperClientModule.
 import org.apache.aurora.scheduler.zookeeper.guice.client.flagged.FlaggedClientConfig;
 
 import static org.apache.aurora.common.logging.RootLogConfig.Configuration;
+import static org.apache.aurora.gen.apiConstants.THRIFT_API_VERSION;
 
 /**
  * Launcher for the aurora scheduler.
  */
-public class SchedulerMain implements Application {
+public class SchedulerMain {
   private static final Logger LOG = Logger.getLogger(SchedulerMain.class.getName());
 
   @NotNull
@@ -138,93 +148,6 @@ public class SchedulerMain implements Application {
   @AppStartup
   private ServiceManagerIface startupServices;
 
-  private static Iterable<? extends Module> getExtraModules() {
-    Builder<Module> modules = ImmutableList.builder();
-
-    for (Class<? extends Module> moduleClass : EXTRA_MODULES.get()) {
-      modules.add(Modules.getModule(moduleClass));
-    }
-
-    return modules.build();
-  }
-
-  @VisibleForTesting
-  Iterable<? extends Module> getModules(
-      String clusterName,
-      String serverSetPath,
-      ClientConfig zkClientConfig,
-      String statsURLPrefix) {
-
-    return ImmutableList.<Module>builder()
-        .add(new StatsModule())
-        .add(new AppModule(clusterName, serverSetPath, zkClientConfig, statsURLPrefix))
-        .addAll(getExtraModules())
-        .add(getPersistentStorageModule())
-        .add(new CronModule())
-        .add(DbModule.productionModule(Bindings.annotatedKeyFactory(Storage.Volatile.class)))
-        .add(new DbModule.GarbageCollectorModule())
-        .build();
-  }
-
-  protected Module getPersistentStorageModule() {
-    return new AbstractModule() {
-      @Override
-      protected void configure() {
-        install(new LogStorageModule());
-      }
-    };
-  }
-
-  protected Module getMesosModules() {
-    final ClientConfig zkClientConfig = FlaggedClientConfig.create();
-    return new AbstractModule() {
-      @Override
-      protected void configure() {
-        install(new CommandLineDriverSettingsModule());
-        install(new LibMesosLoadingModule());
-        install(new MesosLogStreamModule(zkClientConfig));
-      }
-    };
-  }
-
-  @Override
-  public Iterable<? extends Module> getModules() {
-    ClientConfig zkClientConfig = FlaggedClientConfig.create();
-    return ImmutableList.<Module>builder()
-        .add(new LifecycleModule())
-        .add(new AppLauncherModule())
-        .add(new BackupModule(SnapshotStoreImpl.class))
-        .addAll(
-            getModules(
-                CLUSTER_NAME.get(),
-                SERVERSET_PATH.get(),
-                zkClientConfig,
-                STATS_URL_PREFIX.get()))
-        .add(new ZooKeeperClientModule(zkClientConfig))
-        .add(new AbstractModule() {
-          @Override
-          protected void configure() {
-            ResourceSlot executorOverhead = new ResourceSlot(
-                EXECUTOR_OVERHEAD_CPUS.get(),
-                EXECUTOR_OVERHEAD_RAM.get(),
-                Amount.of(0L, Data.MB),
-                0);
-
-            bind(ExecutorSettings.class)
-                .toInstance(ExecutorSettings.newBuilder()
-                    .setExecutorPath(THERMOS_EXECUTOR_PATH.get())
-                    .setExecutorResources(THERMOS_EXECUTOR_RESOURCES.get())
-                    .setThermosObserverRoot(THERMOS_OBSERVER_ROOT.get())
-                    .setExecutorFlags(Optional.fromNullable(THERMOS_EXECUTOR_FLAGS.get()))
-                    .setExecutorOverhead(executorOverhead)
-                    .setGlobalContainerMounts(GLOBAL_CONTAINER_MOUNTS.get())
-                    .build());
-          }
-        })
-        .add(getMesosModules())
-        .build();
-  }
-
   private void stop() {
     LOG.info("Stopping scheduler services.");
     try {
@@ -235,7 +158,7 @@ public class SchedulerMain implements Application {
     appLifecycle.shutdown();
   }
 
-  public void run() {
+  void run() {
     startupServices.startAsync();
     Runtime.getRuntime().addShutdownHook(new Thread(SchedulerMain.this::stop, "ShutdownHook"));
     startupServices.awaitHealthy();
@@ -267,7 +190,122 @@ public class SchedulerMain implements Application {
     stop();
   }
 
+  @VisibleForTesting
+  static Module getUniversalModule() {
+    return Modules.combine(
+        new LifecycleModule(),
+        new StatsModule(),
+        new AppModule(),
+        new CronModule(),
+        DbModule.productionModule(Bindings.annotatedKeyFactory(Storage.Volatile.class)),
+        new DbModule.GarbageCollectorModule());
+  }
+
+  /**
+   * Runs the scheduler by including modules configured from command line arguments in
+   * addition to the provided environment-specific module.
+   *
+   * @param appEnvironmentModule Additional modules based on the execution environment.
+   */
+  @VisibleForTesting
+  public static void flagConfiguredMain(Module appEnvironmentModule) {
+    AtomicLong uncaughtExceptions = Stats.exportLong("uncaught_exceptions");
+    Thread.setDefaultUncaughtExceptionHandler(new UncaughtExceptionHandler() {
+      @Override
+      public void uncaughtException(Thread t, Throwable e) {
+        uncaughtExceptions.incrementAndGet();
+        LOG.log(Level.SEVERE, "Uncaught exception from " + t + ":" + e, e);
+      }
+    });
+
+    ClientConfig zkClientConfig = FlaggedClientConfig.create();
+    Module module = Modules.combine(
+        appEnvironmentModule,
+        getUniversalModule(),
+        new ZooKeeperClientModule(zkClientConfig),
+        new ServiceDiscoveryModule(SERVERSET_PATH.get(), zkClientConfig.credentials),
+        new BackupModule(SnapshotStoreImpl.class),
+        new AbstractModule() {
+          @Override
+          protected void configure() {
+            ResourceSlot executorOverhead = new ResourceSlot(
+                EXECUTOR_OVERHEAD_CPUS.get(),
+                EXECUTOR_OVERHEAD_RAM.get(),
+                Amount.of(0L, Data.MB),
+                0);
+
+            bind(ExecutorSettings.class)
+                .toInstance(ExecutorSettings.newBuilder()
+                    .setExecutorPath(THERMOS_EXECUTOR_PATH.get())
+                    .setExecutorResources(THERMOS_EXECUTOR_RESOURCES.get())
+                    .setThermosObserverRoot(THERMOS_OBSERVER_ROOT.get())
+                    .setExecutorFlags(Optional.fromNullable(THERMOS_EXECUTOR_FLAGS.get()))
+                    .setExecutorOverhead(executorOverhead)
+                    .setGlobalContainerMounts(GLOBAL_CONTAINER_MOUNTS.get())
+                    .build());
+
+            bind(IServerInfo.class).toInstance(
+                IServerInfo.build(
+                    new ServerInfo()
+                        .setClusterName(CLUSTER_NAME.get())
+                        .setThriftAPIVersion(THRIFT_API_VERSION)
+                        .setStatsUrlPrefix(STATS_URL_PREFIX.get())));
+          }
+        });
+
+    Lifecycle lifecycle = null;
+    try {
+      Injector injector = Guice.createInjector(module);
+      lifecycle = injector.getInstance(Lifecycle.class);
+      SchedulerMain scheduler = new SchedulerMain();
+      injector.injectMembers(scheduler);
+      try {
+        scheduler.run();
+      } finally {
+        LOG.info("Application run() exited.");
+      }
+    } finally {
+      if (lifecycle != null) {
+        lifecycle.shutdown();
+      }
+    }
+  }
+
   public static void main(String... args) {
-    AppLauncher.launch(SchedulerMain.class, args);
+    applyStaticArgumentValues(args);
+
+    List<Module> modules = ImmutableList.<Module>builder()
+        .add(
+            new CommandLineDriverSettingsModule(),
+            new LibMesosLoadingModule(),
+            new MesosLogStreamModule(FlaggedClientConfig.create()),
+            new LogStorageModule())
+        .addAll(Iterables.transform(EXTRA_MODULES.get(), MoreModules::getModule))
+        .build();
+    flagConfiguredMain(Modules.combine(modules));
+  }
+
+  private static void exit(String message, Exception error) {
+    LOG.log(Level.SEVERE, message + "\n" + error, error);
+    System.exit(1);
+  }
+
+  /**
+   * Applies {@link CmdLine} arg values throughout the classpath.  This must be invoked before
+   * attempting to read any argument values in the system.
+   *
+   * @param args Command line arguments.
+   */
+  @VisibleForTesting
+  public static void applyStaticArgumentValues(String... args) {
+    try {
+      if (!new ArgScanner().parse(Arrays.asList(args))) {
+        System.exit(0);
+      }
+    } catch (ArgScanException e) {
+      exit("Failed to scan arguments", e);
+    } catch (IllegalArgumentException e) {
+      exit("Failed to apply arguments", e);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/main/java/org/apache/aurora/scheduler/app/ServiceDiscoveryModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/ServiceDiscoveryModule.java b/src/main/java/org/apache/aurora/scheduler/app/ServiceDiscoveryModule.java
new file mode 100644
index 0000000..078042b
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/app/ServiceDiscoveryModule.java
@@ -0,0 +1,91 @@
+/**
+ * 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.app;
+
+import java.util.List;
+import java.util.logging.Logger;
+
+import javax.inject.Singleton;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Provides;
+
+import org.apache.aurora.common.net.pool.DynamicHostSet;
+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.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.zookeeper.data.ACL;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Binding module for utilities to advertise the network presence of the scheduler.
+ */
+public class ServiceDiscoveryModule extends AbstractModule {
+
+  private static final Logger LOG = Logger.getLogger(ServiceDiscoveryModule.class.getName());
+
+  private final String serverSetPath;
+  private final Credentials zkCredentials;
+
+  public ServiceDiscoveryModule(String serverSetPath, Credentials zkCredentials) {
+    this.serverSetPath = requireNonNull(serverSetPath);
+    this.zkCredentials = requireNonNull(zkCredentials);
+  }
+
+  @Override
+  protected void configure() {
+    // provider-only module.
+  }
+
+  @Provides
+  @Singleton
+  List<ACL> provideAcls() {
+    if (zkCredentials == Credentials.NONE) {
+      LOG.warning("Running without ZooKeeper digest credentials. ZooKeeper ACLs are disabled.");
+      return ZooKeeperUtils.OPEN_ACL_UNSAFE;
+    } else {
+      return ZooKeeperUtils.EVERYONE_READ_CREATOR_ALL;
+    }
+  }
+
+  @Provides
+  @Singleton
+  ServerSet provideServerSet(ZooKeeperClient client, List<ACL> zooKeeperAcls) {
+    return new ServerSetImpl(client, zooKeeperAcls, serverSetPath);
+  }
+
+  @Provides
+  @Singleton
+  DynamicHostSet<ServiceInstance> provideSchedulerHostSet(ServerSet serverSet) {
+    // Used for a type re-binding of the serverset.
+    return serverSet;
+  }
+
+  @Provides
+  @Singleton
+  SingletonService provideSingletonService(
+      ZooKeeperClient client,
+      ServerSet serverSet,
+      List<ACL> zookeeperAcls) {
+
+    return new SingletonService(
+        serverSet,
+        SingletonService.createSingletonCandidate(client, serverSetPath, zookeeperAcls));
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java b/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java
index f317b68..4b6a872 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityModule.java
@@ -38,7 +38,7 @@ import org.apache.aurora.common.args.Arg;
 import org.apache.aurora.common.args.CmdLine;
 import org.apache.aurora.gen.AuroraAdmin;
 import org.apache.aurora.gen.AuroraSchedulerManager;
-import org.apache.aurora.scheduler.app.Modules;
+import org.apache.aurora.scheduler.app.MoreModules;
 import org.apache.aurora.scheduler.thrift.aop.AnnotatedAuroraAdmin;
 import org.apache.shiro.SecurityUtils;
 import org.apache.shiro.guice.aop.ShiroAopModule;
@@ -74,7 +74,7 @@ public class HttpSecurityModule extends ServletModule {
   @CmdLine(name = "shiro_realm_modules",
       help = "Guice modules for configuring Shiro Realms.")
   private static final Arg<Set<Module>> SHIRO_REALM_MODULE = Arg.create(
-      ImmutableSet.of(Modules.lazilyInstantiated(IniShiroRealmModule.class)));
+      ImmutableSet.of(MoreModules.lazilyInstantiated(IniShiroRealmModule.class)));
 
   @VisibleForTesting
   static final Matcher<Method> AURORA_SCHEDULER_MANAGER_SERVICE =

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/main/java/org/apache/aurora/scheduler/http/api/security/ModuleParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/security/ModuleParser.java b/src/main/java/org/apache/aurora/scheduler/http/api/security/ModuleParser.java
index 63ab955..ccd9a20 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/security/ModuleParser.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/security/ModuleParser.java
@@ -20,7 +20,7 @@ import com.google.inject.Module;
 
 import org.apache.aurora.common.args.ArgParser;
 import org.apache.aurora.common.args.parsers.NonParameterizedTypeParser;
-import org.apache.aurora.scheduler.app.Modules;
+import org.apache.aurora.scheduler.app.MoreModules;
 
 /**
  * ArgParser for Guice modules. Constructs an instance of a Module with a given alias or FQCN if it
@@ -49,6 +49,6 @@ public class ModuleParser extends NonParameterizedTypeParser<Module> {
     @SuppressWarnings("unchecked")
     Class<? extends Module> moduleClass = (Class<? extends Module>) rawClass;
 
-    return Modules.lazilyInstantiated(moduleClass);
+    return MoreModules.lazilyInstantiated(moduleClass);
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java b/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java
index 3ccee66..e1a21f6 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/auth/ThriftAuthModule.java
@@ -30,7 +30,7 @@ import org.apache.aurora.auth.UnsecureAuthModule;
 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.scheduler.app.Modules;
+import org.apache.aurora.scheduler.app.MoreModules;
 
 import static java.util.Objects.requireNonNull;
 
@@ -78,6 +78,6 @@ public class ThriftAuthModule extends AbstractModule {
     requireBinding(SessionValidator.class);
     requireBinding(CapabilityValidator.class);
 
-    install(Modules.wrapInPrivateModule(authModule, AUTH_MODULE_CLASSES));
+    install(MoreModules.wrapInPrivateModule(authModule, AUTH_MODULE_CLASSES));
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java b/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java
index d9f7b20..bab4567 100644
--- a/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java
@@ -13,7 +13,6 @@
  */
 package org.apache.aurora.scheduler;
 
-import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.aurora.GuavaUtils.ServiceManagerIface;
@@ -82,12 +81,7 @@ public class SchedulerLifecycleTest extends EasyMockTest {
 
     schedulerLifecycle = new SchedulerLifecycle(
         storageUtil.storage,
-        new Lifecycle(shutdownRegistry, new UncaughtExceptionHandler() {
-          @Override
-          public void uncaughtException(Thread t, Throwable e) {
-            fail(e.getMessage());
-          }
-        }),
+        new Lifecycle(shutdownRegistry),
         driver,
         delayedActions,
         eventSink,

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/test/java/org/apache/aurora/scheduler/app/ModulesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/app/ModulesTest.java b/src/test/java/org/apache/aurora/scheduler/app/ModulesTest.java
deleted file mode 100644
index 01137e9..0000000
--- a/src/test/java/org/apache/aurora/scheduler/app/ModulesTest.java
+++ /dev/null
@@ -1,39 +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.scheduler.app;
-
-import com.google.inject.AbstractModule;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class ModulesTest {
-  private static final String STRING = "string";
-
-  @Test
-  public void testLazilyInstantiated() {
-    Injector injector = Guice.createInjector(Modules.lazilyInstantiated(StringInstaller.class));
-    assertEquals(STRING, injector.getInstance(String.class));
-  }
-
-  static class StringInstaller extends AbstractModule {
-    @Override
-    protected void configure() {
-      bind(String.class).toInstance(STRING);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/test/java/org/apache/aurora/scheduler/app/MoreModulesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/app/MoreModulesTest.java b/src/test/java/org/apache/aurora/scheduler/app/MoreModulesTest.java
new file mode 100644
index 0000000..b2fb3c9
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/app/MoreModulesTest.java
@@ -0,0 +1,39 @@
+/**
+ * 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.app;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class MoreModulesTest {
+  private static final String STRING = "string";
+
+  @Test
+  public void testLazilyInstantiated() {
+    Injector injector = Guice.createInjector(MoreModules.lazilyInstantiated(StringInstaller.class));
+    assertEquals(STRING, injector.getInstance(String.class));
+  }
+
+  static class StringInstaller extends AbstractModule {
+    @Override
+    protected void configure() {
+      bind(String.class).toInstance(STRING);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/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 a44b9da..1530ef8 100644
--- a/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
@@ -46,7 +46,6 @@ import com.google.inject.Module;
 import org.apache.aurora.GuavaUtils;
 import org.apache.aurora.codec.ThriftBinaryCodec.CodingException;
 import org.apache.aurora.common.application.Lifecycle;
-import org.apache.aurora.common.application.modules.AppLauncherModule;
 import org.apache.aurora.common.net.pool.DynamicHostSet.HostChangeMonitor;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Data;
@@ -55,12 +54,14 @@ 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.ZooKeeperClient.Credentials;
 import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest;
 import org.apache.aurora.gen.AssignedTask;
 import org.apache.aurora.gen.Identity;
 import org.apache.aurora.gen.JobKey;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.gen.ScheduledTask;
+import org.apache.aurora.gen.ServerInfo;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskEvent;
 import org.apache.aurora.gen.storage.LogEntry;
@@ -80,6 +81,7 @@ import org.apache.aurora.scheduler.mesos.DriverFactory;
 import org.apache.aurora.scheduler.mesos.DriverSettings;
 import org.apache.aurora.scheduler.mesos.ExecutorSettings;
 import org.apache.aurora.scheduler.storage.backup.BackupModule;
+import org.apache.aurora.scheduler.storage.entities.IServerInfo;
 import org.apache.aurora.scheduler.storage.log.EntrySerializer;
 import org.apache.aurora.scheduler.storage.log.LogStorageModule;
 import org.apache.aurora.scheduler.storage.log.SnapshotStoreImpl;
@@ -100,6 +102,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import static org.apache.aurora.common.testing.easymock.EasyMockTest.createCapture;
+import static org.apache.aurora.gen.apiConstants.THRIFT_API_VERSION;
 import static org.apache.mesos.Protos.FrameworkInfo;
 import static org.easymock.EasyMock.capture;
 import static org.easymock.EasyMock.createControl;
@@ -202,25 +205,32 @@ public class SchedulerIT extends BaseZooKeeperTest {
                 .setExecutorOverhead(executorOverhead)
                 .build());
         install(new BackupModule(backupDir, SnapshotStoreImpl.class));
+
+        bind(IServerInfo.class).toInstance(
+            IServerInfo.build(
+                new ServerInfo()
+                    .setClusterName(CLUSTER_NAME)
+                    .setThriftAPIVersion(THRIFT_API_VERSION)
+                    .setStatsUrlPrefix(STATS_URL_PREFIX)));
       }
     };
+    Credentials credentials = ZooKeeperClient.digestCredentials("mesos", "mesos");
     ClientConfig zkClientConfig = ClientConfig
         .create(ImmutableList.of(InetSocketAddress.createUnresolved("localhost", getPort())))
-        .withCredentials(ZooKeeperClient.digestCredentials("mesos", "mesos"));
-    final SchedulerMain main = SchedulerMain.class.newInstance();
+        .withCredentials(credentials);
+    SchedulerMain main = SchedulerMain.class.newInstance();
     injector = Guice.createInjector(
         ImmutableList.<Module>builder()
-            .addAll(main.getModules(CLUSTER_NAME, SERVERSET_PATH, zkClientConfig, STATS_URL_PREFIX))
-            .add(new LifecycleModule())
-            .add(new AppLauncherModule())
+            .add(SchedulerMain.getUniversalModule())
+            .add(new LogStorageModule())
             .add(new ZooKeeperClientModule(zkClientConfig))
+            .add(new ServiceDiscoveryModule(SERVERSET_PATH, credentials))
             .add(testModule)
             .build()
     );
     injector.injectMembers(main);
     lifecycle = injector.getInstance(Lifecycle.class);
 
-    // Mimic AppLauncher running main.
     executor.submit(new Runnable() {
       @Override
       public void run() {

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/test/java/org/apache/aurora/scheduler/app/local/LocalSchedulerMain.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/app/local/LocalSchedulerMain.java b/src/test/java/org/apache/aurora/scheduler/app/local/LocalSchedulerMain.java
index 738c8b6..4716b66 100644
--- a/src/test/java/org/apache/aurora/scheduler/app/local/LocalSchedulerMain.java
+++ b/src/test/java/org/apache/aurora/scheduler/app/local/LocalSchedulerMain.java
@@ -24,10 +24,8 @@ import com.google.common.io.Files;
 import com.google.inject.AbstractModule;
 import com.google.inject.Key;
 import com.google.inject.Module;
+import com.google.inject.util.Modules;
 
-import org.apache.aurora.codec.ThriftBinaryCodec.CodingException;
-import org.apache.aurora.common.application.AppLauncher;
-import org.apache.aurora.gen.storage.Snapshot;
 import org.apache.aurora.scheduler.app.SchedulerMain;
 import org.apache.aurora.scheduler.app.local.simulator.ClusterSimulatorModule;
 import org.apache.aurora.scheduler.mesos.DriverFactory;
@@ -42,7 +40,10 @@ import org.apache.shiro.io.ResourceUtils;
 /**
  * A main class that runs the scheduler in local mode, using fakes for external components.
  */
-public class LocalSchedulerMain extends SchedulerMain {
+public final class LocalSchedulerMain {
+  private LocalSchedulerMain() {
+    // Utility class.
+  }
 
   private static final DriverSettings DRIVER_SETTINGS = new DriverSettings(
       "fakemaster",
@@ -52,41 +53,9 @@ public class LocalSchedulerMain extends SchedulerMain {
           .setName("test framework")
           .build());
 
-  @Override
-  protected Module getPersistentStorageModule() {
-    return new AbstractModule() {
-      @Override
-      protected void configure() {
-        bind(Storage.class).to(Key.get(Storage.class, Storage.Volatile.class));
-        bind(NonVolatileStorage.class).to(FakeNonVolatileStorage.class);
-        bind(DistributedSnapshotStore.class).toInstance(new DistributedSnapshotStore() {
-          @Override
-          public void persist(Snapshot snapshot) throws CodingException {
-            // No-op.
-          }
-        });
-      }
-    };
-  }
-
-  @Override
-  protected Module getMesosModules() {
-    return new AbstractModule() {
-      @Override
-      protected void configure() {
-        bind(DriverSettings.class).toInstance(DRIVER_SETTINGS);
-        bind(SchedulerDriver.class).to(FakeMaster.class);
-        bind(DriverFactory.class).to(FakeMaster.class);
-        bind(FakeMaster.class).in(Singleton.class);
-        install(new ClusterSimulatorModule());
-      }
-    };
-  }
-
   public static void main(String[] args) {
     File backupDir = Files.createTempDir();
     backupDir.deleteOnExit();
-
     List<String> arguments = ImmutableList.<String>builder()
         .add(args)
         .add("-cluster_name=local")
@@ -103,7 +72,28 @@ public class LocalSchedulerMain extends SchedulerMain {
             + "org/apache/aurora/scheduler/http/api/security/shiro-example.ini")
         .add("-enable_h2_console=true")
         .build();
+    SchedulerMain.applyStaticArgumentValues(arguments.toArray(new String[] {}));
+
+    Module persistentStorage = new AbstractModule() {
+      @Override
+      protected void configure() {
+        bind(Storage.class).to(Key.get(Storage.class, Storage.Volatile.class));
+        bind(NonVolatileStorage.class).to(FakeNonVolatileStorage.class);
+        bind(DistributedSnapshotStore.class).toInstance(snapshot -> { });
+      }
+    };
+
+    Module fakeMesos = new AbstractModule() {
+      @Override
+      protected void configure() {
+        bind(DriverSettings.class).toInstance(DRIVER_SETTINGS);
+        bind(SchedulerDriver.class).to(FakeMaster.class);
+        bind(DriverFactory.class).to(FakeMaster.class);
+        bind(FakeMaster.class).in(Singleton.class);
+        install(new ClusterSimulatorModule());
+      }
+    };
 
-    AppLauncher.launch(LocalSchedulerMain.class, arguments.toArray(new String[0]));
+    SchedulerMain.flagConfiguredMain(Modules.combine(fakeMesos, persistentStorage));
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/test/java/org/apache/aurora/scheduler/http/QuitCallbackTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/QuitCallbackTest.java b/src/test/java/org/apache/aurora/scheduler/http/QuitCallbackTest.java
index e50bdc5..62837c7 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/QuitCallbackTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/QuitCallbackTest.java
@@ -13,12 +13,9 @@
  */
 package org.apache.aurora.scheduler.http;
 
-import java.lang.Thread.UncaughtExceptionHandler;
-
 import org.apache.aurora.common.application.Lifecycle;
 import org.apache.aurora.common.base.Command;
 import org.apache.aurora.common.testing.easymock.EasyMockTest;
-
 import org.junit.Before;
 import org.junit.Test;
 
@@ -30,8 +27,7 @@ public class QuitCallbackTest extends EasyMockTest {
   @Before
   public void setUp() {
     shutdownCommand = createMock(Command.class);
-    handler = new QuitCallback(
-        new Lifecycle(shutdownCommand, createMock(UncaughtExceptionHandler.class)));
+    handler = new QuitCallback(new Lifecycle(shutdownCommand));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/test/java/org/apache/aurora/scheduler/log/mesos/MesosLogTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/log/mesos/MesosLogTest.java b/src/test/java/org/apache/aurora/scheduler/log/mesos/MesosLogTest.java
index cb33b03..9189bde 100644
--- a/src/test/java/org/apache/aurora/scheduler/log/mesos/MesosLogTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/log/mesos/MesosLogTest.java
@@ -79,7 +79,7 @@ public class MesosLogTest extends EasyMockTest {
             .toInstance(WRITE_TIMEOUT);
         bind(byte[].class).annotatedWith(MesosLog.NoopEntry.class)
             .toInstance(DUMMY_CONTENT.getBytes(StandardCharsets.UTF_8));
-        bind(Lifecycle.class).toInstance(new Lifecycle(shutdownHooks, null));
+        bind(Lifecycle.class).toInstance(new Lifecycle(shutdownHooks));
       }
     });
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java b/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java
index 7e1d13b..ecef202 100644
--- a/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java
@@ -13,7 +13,6 @@
  */
 package org.apache.aurora.scheduler.mesos;
 
-import java.lang.Thread.UncaughtExceptionHandler;
 import java.nio.charset.StandardCharsets;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.logging.Level;
@@ -153,15 +152,13 @@ public class MesosSchedulerImplTest extends EasyMockTest {
   private void initializeScheduler(Logger logger) {
     storageUtil = new StorageTestUtil(this);
     shutdownCommand = createMock(Command.class);
-    final Lifecycle lifecycle =
-        new Lifecycle(shutdownCommand, createMock(UncaughtExceptionHandler.class));
     statusHandler = createMock(TaskStatusHandler.class);
     offerManager = createMock(OfferManager.class);
     eventSink = createMock(EventSink.class);
 
     scheduler = new MesosSchedulerImpl(
         storageUtil.storage,
-        lifecycle,
+        new Lifecycle(shutdownCommand),
         statusHandler,
         offerManager,
         eventSink,

http://git-wip-us.apache.org/repos/asf/aurora/blob/44e47264/src/test/java/org/apache/aurora/scheduler/sla/SlaModuleTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/sla/SlaModuleTest.java b/src/test/java/org/apache/aurora/scheduler/sla/SlaModuleTest.java
index 3e9ef10..00a872c 100644
--- a/src/test/java/org/apache/aurora/scheduler/sla/SlaModuleTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/sla/SlaModuleTest.java
@@ -26,7 +26,6 @@ import com.google.inject.Injector;
 import com.google.inject.Key;
 import com.google.inject.Module;
 
-import org.apache.aurora.common.application.modules.AppLauncherModule;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.stats.Stat;
@@ -73,7 +72,6 @@ public class SlaModuleTest extends EasyMockTest {
         ImmutableList.<Module>builder()
             .add(module)
             .add(new LifecycleModule())
-            .add(new AppLauncherModule())
             .add(new AbstractModule() {
               @Override
               protected void configure() {