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/04/09 06:51:37 UTC

aurora git commit: Remove use of LocalServiceRegistry, simplify plumbing of HTTP address.

Repository: aurora
Updated Branches:
  refs/heads/master 218457b98 -> 165c4dff7


Remove use of LocalServiceRegistry, simplify plumbing of HTTP address.

Testing Done:
Test suite + end-to-end tests.

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


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

Branch: refs/heads/master
Commit: 165c4dff79d488cb3b0757e94fed7e242e2ce107
Parents: 218457b
Author: Bill Farner <wf...@apache.org>
Authored: Wed Apr 8 21:47:29 2015 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Wed Apr 8 21:47:29 2015 -0700

----------------------------------------------------------------------
 .../apache/aurora/scheduler/app/AppModule.java  | 10 +-
 .../app/LocalServiceRegistryWithOverrides.java  | 91 ------------------
 .../aurora/scheduler/app/SchedulerMain.java     | 35 +++----
 .../aurora/scheduler/http/HttpService.java      | 29 ++++++
 .../scheduler/http/JettyServerModule.java       | 98 +++++++++++++++-----
 .../aurora/scheduler/http/LeaderRedirect.java   | 18 ++--
 .../LocalServiceRegistryWithOverridesTest.java  | 64 -------------
 .../aurora/scheduler/app/SchedulerIT.java       |  8 +-
 .../scheduler/http/JettyServerModuleTest.java   | 16 +---
 .../scheduler/http/LeaderRedirectTest.java      | 26 +-----
 .../scheduler/http/ServletFilterTest.java       |  2 +-
 .../http/api/security/ApiSecurityIT.java        |  2 +-
 12 files changed, 135 insertions(+), 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/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 5f6a019..d2c1720 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
@@ -22,7 +22,6 @@ import javax.inject.Singleton;
 
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
-import com.google.common.base.Optional;
 import com.google.common.collect.Iterables;
 import com.google.inject.AbstractModule;
 import com.google.inject.Provides;
@@ -79,20 +78,17 @@ public class AppModule extends AbstractModule {
   private final String serverSetPath;
   private final ClientConfig zkClientConfig;
   private final String statsUrlPrefix;
-  private final Optional<String> zkLocalDnsNameOverride;
 
   AppModule(
       String clusterName,
       String serverSetPath,
       ClientConfig zkClientConfig,
-      String statsUrlPrefix,
-      Optional<String> zkLocalDnsNameOverride) {
+      String statsUrlPrefix) {
 
     this.clusterName = checkNotBlank(clusterName);
     this.serverSetPath = checkNotBlank(serverSetPath);
     this.zkClientConfig = requireNonNull(zkClientConfig);
     this.statsUrlPrefix = statsUrlPrefix;
-    this.zkLocalDnsNameOverride = requireNonNull(zkLocalDnsNameOverride);
   }
 
   @Override
@@ -130,10 +126,6 @@ public class AppModule extends AbstractModule {
     install(new StateModule());
     install(new SlaModule());
     install(new UpdaterModule());
-
-    bind(LocalServiceRegistryWithOverrides.Settings.class).toInstance(
-        new LocalServiceRegistryWithOverrides.Settings(this.zkLocalDnsNameOverride)
-    );
     bind(StatsProvider.class).toInstance(Stats.STATS_PROVIDER);
   }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/main/java/org/apache/aurora/scheduler/app/LocalServiceRegistryWithOverrides.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/LocalServiceRegistryWithOverrides.java b/src/main/java/org/apache/aurora/scheduler/app/LocalServiceRegistryWithOverrides.java
deleted file mode 100644
index 6d92ae3..0000000
--- a/src/main/java/org/apache/aurora/scheduler/app/LocalServiceRegistryWithOverrides.java
+++ /dev/null
@@ -1,91 +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 java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.util.Map;
-
-import com.google.common.base.Function;
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
-import com.google.inject.Inject;
-import com.twitter.common.application.modules.LocalServiceRegistry;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Wraps the provided LocalServiceRegistry and optionally overrides the hostname it provides.
- */
-public class LocalServiceRegistryWithOverrides {
-  public static class Settings {
-    private final Optional<String> zkLocalDnsNameOverride;
-
-    public Settings(Optional<String> zkLocalDnsNameOverride) {
-      if (zkLocalDnsNameOverride.isPresent()) {
-        /* Force resolution of the DNS address passed in to ensure it's valid */
-        try {
-          InetAddress.getByName(zkLocalDnsNameOverride.get());
-        } catch (UnknownHostException e) {
-          throw new IllegalStateException(
-              "Failed to resolve hostname supplied by -hostname", e);
-        }
-      }
-      this.zkLocalDnsNameOverride = zkLocalDnsNameOverride;
-    }
-
-    public Optional<String> getZkLocalDnsNameOverride() {
-      return zkLocalDnsNameOverride;
-    }
-  }
-
-  private final LocalServiceRegistry wrapped;
-  private final Optional<String> zkLocalDnsNameOverride;
-
-  @Inject
-  public LocalServiceRegistryWithOverrides(
-      LocalServiceRegistry registry,
-      Settings settings) {
-    this.wrapped = requireNonNull(registry);
-    this.zkLocalDnsNameOverride = settings.getZkLocalDnsNameOverride();
-  }
-
-  private Map<String, InetSocketAddress> applyDnsOverrides(
-      Map<String, InetSocketAddress> services) {
-    final InetAddress inetAddress;
-    try {
-      inetAddress = InetAddress.getByName(this.zkLocalDnsNameOverride.get());
-    } catch (UnknownHostException e) {
-      throw new RuntimeException("Failed to resolve address.", e);
-    }
-    return ImmutableMap.copyOf(
-        Maps.transformValues(services, new Function<InetSocketAddress, InetSocketAddress>() {
-          @Override
-          public InetSocketAddress apply(InetSocketAddress input) {
-            return new InetSocketAddress(inetAddress, input.getPort());
-          }
-        }));
-  }
-
-  public Map<String, InetSocketAddress> getAuxiliarySockets() {
-    Map<String, InetSocketAddress> auxSockets = wrapped.getAuxiliarySockets();
-    if (zkLocalDnsNameOverride.isPresent()) {
-      return applyDnsOverrides(auxSockets);
-    } else {
-      return auxSockets;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/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 2af009d..087abe5 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
@@ -23,6 +23,8 @@ 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.net.HostAndPort;
 import com.google.inject.AbstractModule;
 import com.google.inject.Module;
 import com.twitter.common.application.AbstractApplication;
@@ -48,6 +50,7 @@ import org.apache.aurora.gen.Volume;
 import org.apache.aurora.scheduler.SchedulerLifecycle;
 import org.apache.aurora.scheduler.configuration.Resources;
 import org.apache.aurora.scheduler.cron.quartz.CronModule;
+import org.apache.aurora.scheduler.http.HttpService;
 import org.apache.aurora.scheduler.log.mesos.MesosLogStreamModule;
 import org.apache.aurora.scheduler.mesos.CommandLineDriverSettingsModule;
 import org.apache.aurora.scheduler.mesos.ExecutorSettings;
@@ -121,10 +124,6 @@ public class SchedulerMain extends AbstractApplication {
   @CmdLine(name = "viz_job_url_prefix", help = "URL prefix for job container stats.")
   private static final Arg<String> STATS_URL_PREFIX = Arg.create("");
 
-  @CmdLine(name = "hostname",
-      help = "The hostname to advertise in ZooKeeper instead of the locally-resolved hostname.")
-  private static final Arg<String> HOSTNAME_OVERRIDE = Arg.create(null);
-
   @CmdLine(name = "global_container_mounts",
       help = "A comma seperated list of mount points (in host:container form) to mount "
           + "into all (non-mesos) containers.")
@@ -132,7 +131,7 @@ public class SchedulerMain extends AbstractApplication {
       Arg.<List<Volume>>create(ImmutableList.<Volume>of());
 
   @Inject private SingletonService schedulerService;
-  @Inject private LocalServiceRegistryWithOverrides serviceRegistry;
+  @Inject private HttpService httpService;
   @Inject private SchedulerLifecycle schedulerLifecycle;
   @Inject private Lifecycle appLifecycle;
 
@@ -151,17 +150,11 @@ public class SchedulerMain extends AbstractApplication {
       String clusterName,
       String serverSetPath,
       ClientConfig zkClientConfig,
-      String statsURLPrefix,
-      Optional<String> zkLocalDnsNameOverride) {
+      String statsURLPrefix) {
 
     return ImmutableList.<Module>builder()
         .add(new StatsModule())
-        .add(new AppModule(
-            clusterName,
-            serverSetPath,
-            zkClientConfig,
-            statsURLPrefix,
-            zkLocalDnsNameOverride))
+        .add(new AppModule(clusterName, serverSetPath, zkClientConfig, statsURLPrefix))
         .addAll(getExtraModules())
         .add(getPersistentStorageModule())
         .add(new MemStorageModule(Bindings.annotatedKeyFactory(LogStorage.WriteBehind.class)))
@@ -205,8 +198,7 @@ public class SchedulerMain extends AbstractApplication {
                 CLUSTER_NAME.get(),
                 SERVERSET_PATH.get(),
                 zkClientConfig,
-                STATS_URL_PREFIX.get(),
-                Optional.fromNullable(HOSTNAME_OVERRIDE.get())))
+                STATS_URL_PREFIX.get()))
         .add(new ZooKeeperClientModule(zkClientConfig))
         .add(new AbstractModule() {
           @Override
@@ -256,16 +248,13 @@ public class SchedulerMain extends AbstractApplication {
 
     LeadershipListener leaderListener = schedulerLifecycle.prepare();
 
-    Optional<InetSocketAddress> httpSocket =
-        Optional.fromNullable(serviceRegistry.getAuxiliarySockets().get("http"));
-    if (!httpSocket.isPresent()) {
-      throw new IllegalStateException("No HTTP service registered with LocalServiceRegistry.");
-    }
-
+    HostAndPort httpAddress = httpService.getAddress();
+    InetSocketAddress httpSocketAddress =
+        InetSocketAddress.createUnresolved(httpAddress.getHostText(), httpAddress.getPort());
     try {
       schedulerService.lead(
-          httpSocket.get(),
-          serviceRegistry.getAuxiliarySockets(),
+          httpSocketAddress,
+          ImmutableMap.of("http", httpSocketAddress),
           leaderListener);
     } catch (Group.WatchException e) {
       throw new IllegalStateException("Failed to watch group and lead service.", e);

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/main/java/org/apache/aurora/scheduler/http/HttpService.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/HttpService.java b/src/main/java/org/apache/aurora/scheduler/http/HttpService.java
new file mode 100644
index 0000000..e6e6fca
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/http/HttpService.java
@@ -0,0 +1,29 @@
+/**
+ * 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.http;
+
+import com.google.common.net.HostAndPort;
+
+/**
+ * The HTTP service provided by the application.
+ */
+public interface HttpService {
+
+  /**
+   * Gets the address that can be used by clients to access the service.
+   *
+   * @return Service address.
+   */
+  HostAndPort getAddress();
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java b/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
index 50f3775..16515f6 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/JettyServerModule.java
@@ -13,6 +13,8 @@
  */
 package org.apache.aurora.scheduler.http;
 
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.EnumSet;
 import java.util.Map;
 import java.util.logging.Level;
@@ -25,11 +27,16 @@ import javax.servlet.ServletContextListener;
 import javax.servlet.http.HttpServlet;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.net.HostAndPort;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.inject.AbstractModule;
 import com.google.inject.Injector;
 import com.google.inject.Module;
+import com.google.inject.PrivateModule;
 import com.google.inject.Provides;
 import com.google.inject.TypeLiteral;
 import com.google.inject.name.Names;
@@ -39,11 +46,8 @@ import com.google.inject.util.Modules;
 import com.sun.jersey.api.container.filter.GZIPContentEncodingFilter;
 import com.sun.jersey.guice.JerseyServletModule;
 import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
-import com.twitter.common.application.modules.LifecycleModule;
-import com.twitter.common.application.modules.LifecycleModule.LaunchException;
 import com.twitter.common.args.Arg;
 import com.twitter.common.args.CmdLine;
-import com.twitter.common.base.Command;
 import com.twitter.common.base.ExceptionalSupplier;
 import com.twitter.common.base.MoreSuppliers;
 import com.twitter.common.net.http.handlers.AbortHandler;
@@ -81,7 +85,6 @@ import static java.util.Objects.requireNonNull;
 import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS;
 import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_RESPONSE_FILTERS;
 import static com.sun.jersey.api.json.JSONConfiguration.FEATURE_POJO_MAPPING;
-import static com.twitter.common.application.modules.LocalServiceRegistry.LocalService;
 
 /**
  * Binding module for scheduler HTTP servlets.
@@ -97,7 +100,9 @@ public class JettyServerModule extends AbstractModule {
   // rewritten is stored.
   static final String ORIGINAL_PATH_ATTRIBUTE_NAME = "originalPath";
 
-  public static final String THRIFT_API_PATH_SPEC = "/api";
+  @CmdLine(name = "hostname",
+      help = "The hostname to advertise in ZooKeeper instead of the locally-resolved hostname.")
+  private static final Arg<String> HOSTNAME_OVERRIDE = Arg.create(null);
 
   @Nonnegative
   @CmdLine(name = "http_port",
@@ -140,7 +145,27 @@ public class JettyServerModule extends AbstractModule {
 
     bindConstant().annotatedWith(StringTemplateServlet.CacheTemplates.class).to(true);
 
-    LifecycleModule.bindServiceRunner(binder(), HttpServerLauncher.class);
+    final Optional<String> hostnameOverride = Optional.fromNullable(HOSTNAME_OVERRIDE.get());
+    if (hostnameOverride.isPresent()) {
+        /* Force resolution of the DNS address passed in to ensure it's valid */
+      try {
+        InetAddress.getByName(hostnameOverride.get());
+      } catch (UnknownHostException e) {
+        throw new IllegalStateException(
+            "Failed to resolve hostname supplied by -hostname", e);
+      }
+    }
+    install(new PrivateModule() {
+      @Override
+      protected void configure() {
+        bind(new TypeLiteral<Optional<String>>() { }).toInstance(hostnameOverride);
+        bind(HttpService.class).to(HttpServerLauncher.class);
+        bind(HttpServerLauncher.class).in(Singleton.class);
+        expose(HttpServerLauncher.class);
+        expose(HttpService.class);
+      }
+    });
+    SchedulerServicesModule.addAppStartupServiceBinding(binder()).to(HttpServerLauncher.class);
 
     bind(LeaderRedirect.class).in(Singleton.class);
     SchedulerServicesModule.addAppStartupServiceBinding(binder()).to(RedirectMonitor.class);
@@ -253,13 +278,18 @@ public class JettyServerModule extends AbstractModule {
     }
   }
 
-  // TODO(wfarner): Use guava's Service to enforce the lifecycle of this.
-  public static final class HttpServerLauncher implements LifecycleModule.ServiceRunner {
+  public static final class HttpServerLauncher extends AbstractIdleService implements HttpService {
     private final ServletContextListener servletContextListener;
+    private final Optional<String> advertisedHostOverride;
+    private volatile Server server;
 
     @Inject
-    HttpServerLauncher(ServletContextListener servletContextListener) {
+    HttpServerLauncher(
+        ServletContextListener servletContextListener,
+        Optional<String> advertisedHostOverride) {
+
       this.servletContextListener = requireNonNull(servletContextListener);
+      this.advertisedHostOverride = requireNonNull(advertisedHostOverride);
     }
 
     private static final Map<String, String> REGEX_REWRITE_RULES =
@@ -289,12 +319,35 @@ public class JettyServerModule extends AbstractModule {
     }
 
     @Override
-    public LocalService launch() throws LaunchException {
+    public HostAndPort getAddress() {
+      Preconditions.checkState(state() == State.RUNNING);
+      Connector connector = server.getConnectors()[0];
+
+      String host;
+      if (advertisedHostOverride.isPresent()) {
+        host = advertisedHostOverride.get();
+      } else if (connector.getHost() == null) {
+        // Resolve the local host name.
+        try {
+          host = InetAddress.getLocalHost().getHostAddress();
+        } catch (UnknownHostException e) {
+          throw new RuntimeException("Failed to resolve local host address: " + e, e);
+        }
+      } else {
+        // If jetty was configured with a specific host to bind to, use that.
+        host = connector.getHost();
+      }
+
+      return HostAndPort.fromParts(host, connector.getLocalPort());
+    }
+
+    @Override
+    protected void startUp() {
       // N.B. we explicitly disable the resource cache here due to a bug serving content out of the
       // jar under the vagrant image. C.f. https://bugs.eclipse.org/bugs/show_bug.cgi?id=364936
       Resource.setDefaultUseCaches(false);
 
-      final Server server = new Server();
+      server = new Server();
       ServletContextHandler servletHandler =
           new ServletContextHandler(server, "/", ServletContextHandler.NO_SESSIONS);
 
@@ -319,21 +372,18 @@ public class JettyServerModule extends AbstractModule {
         connector.open();
         server.start();
       } catch (Exception e) {
-        throw new LaunchException("Failed to start jetty server: " + e, e);
+        throw Throwables.propagate(e);
       }
+    }
 
-      Command shutdown = new Command() {
-        @Override public void execute() {
-          LOG.info("Shutting down embedded http server");
-          try {
-            server.stop();
-          } catch (Exception e) {
-            LOG.log(Level.INFO, "Failed to stop jetty server: " + e, e);
-          }
-        }
-      };
-
-      return LocalService.auxiliaryService("http", connector.getLocalPort(), shutdown);
+    @Override
+    protected void shutDown() {
+      LOG.info("Shutting down embedded http server");
+      try {
+        server.stop();
+      } catch (Exception e) {
+        LOG.log(Level.INFO, "Failed to stop jetty server: " + e, e);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirect.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirect.java b/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirect.java
index e03009c..40e0c84 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirect.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/LeaderRedirect.java
@@ -13,8 +13,6 @@
  */
 package org.apache.aurora.scheduler.http;
 
-import java.net.InetSocketAddress;
-import java.util.Objects;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.logging.Logger;
 
@@ -33,7 +31,7 @@ import com.twitter.common.net.pool.DynamicHostSet.MonitorException;
 import com.twitter.thrift.Endpoint;
 import com.twitter.thrift.ServiceInstance;
 
-import org.apache.aurora.scheduler.app.LocalServiceRegistryWithOverrides;
+import static java.util.Objects.requireNonNull;
 
 /**
  * Redirect logic for finding the leading scheduler in the event that this process is not the
@@ -48,17 +46,15 @@ public class LeaderRedirect {
 
   private static final Logger LOG = Logger.getLogger(LeaderRedirect.class.getName());
 
-  private final LocalServiceRegistryWithOverrides serviceRegistry;
+  private final HttpService httpService;
   private final DynamicHostSet<ServiceInstance> schedulers;
 
   private final AtomicReference<ServiceInstance> leader = Atomics.newReference();
 
   @Inject
-  LeaderRedirect(
-      LocalServiceRegistryWithOverrides serviceRegistry,
-      DynamicHostSet<ServiceInstance> schedulers) {
-    this.serviceRegistry = Objects.requireNonNull(serviceRegistry);
-    this.schedulers = Objects.requireNonNull(schedulers);
+  LeaderRedirect(HttpService httpService, DynamicHostSet<ServiceInstance> schedulers) {
+    this.httpService = requireNonNull(httpService);
+    this.schedulers = requireNonNull(schedulers);
   }
 
   /**
@@ -88,9 +84,9 @@ public class LeaderRedirect {
   }
 
   private Optional<HostAndPort> getLocalHttp() {
-    InetSocketAddress localHttp = serviceRegistry.getAuxiliarySockets().get(HTTP_PORT_NAME);
+    HostAndPort localHttp = httpService.getAddress();
     return (localHttp == null) ? Optional.<HostAndPort>absent()
-        : Optional.of(HostAndPort.fromParts(localHttp.getHostName(), localHttp.getPort()));
+        : Optional.of(HostAndPort.fromParts(localHttp.getHostText(), localHttp.getPort()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/test/java/org/apache/aurora/scheduler/app/LocalServiceRegistryWithOverridesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/app/LocalServiceRegistryWithOverridesTest.java b/src/test/java/org/apache/aurora/scheduler/app/LocalServiceRegistryWithOverridesTest.java
deleted file mode 100644
index 21fd027..0000000
--- a/src/test/java/org/apache/aurora/scheduler/app/LocalServiceRegistryWithOverridesTest.java
+++ /dev/null
@@ -1,64 +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 java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.twitter.common.application.modules.LocalServiceRegistry;
-import com.twitter.common.net.InetSocketAddressHelper;
-
-import com.twitter.common.testing.easymock.EasyMockTest;
-
-import org.junit.Test;
-
-import static org.easymock.EasyMock.expect;
-import static org.junit.Assert.assertEquals;
-
-public class LocalServiceRegistryWithOverridesTest extends EasyMockTest {
-  private InetSocketAddress localHost;
-  public LocalServiceRegistryWithOverridesTest() throws UnknownHostException {
-    this.localHost = InetSocketAddressHelper.getLocalAddress(8081);
-  }
-
-  private LocalServiceRegistryWithOverrides createMockServiceRegistry(
-      Optional<String> dnsOverride) throws Exception {
-    LocalServiceRegistry serviceRegistry = control.createMock(LocalServiceRegistry.class);
-    expect(serviceRegistry.getAuxiliarySockets()).andReturn(
-        ImmutableMap.of("http", InetSocketAddressHelper.getLocalAddress(8081)));
-
-    control.replay();
-    return new LocalServiceRegistryWithOverrides(
-        serviceRegistry,
-        new LocalServiceRegistryWithOverrides.Settings(dnsOverride));
-  }
-
-  @Test
-  public void testNoOverride() throws Exception {
-    LocalServiceRegistryWithOverrides registry = createMockServiceRegistry(
-        Optional.<String>absent());
-    InetSocketAddress addr = registry.getAuxiliarySockets().get("http");
-    assertEquals(addr.getHostString(), localHost.getHostString());
-  }
-
-  @Test
-  public void testOverride() throws Exception {
-    LocalServiceRegistryWithOverrides registry = createMockServiceRegistry(
-        Optional.of("www.google.com"));
-    InetSocketAddress addr = registry.getAuxiliarySockets().get("http");
-    assertEquals(addr.getHostString(), "www.google.com");
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/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 d212bbd..7bb1e7a 100644
--- a/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
@@ -216,13 +216,7 @@ public class SchedulerIT extends BaseZooKeeperTest {
     final SchedulerMain main = SchedulerMain.class.newInstance();
     injector = Guice.createInjector(
         ImmutableList.<Module>builder()
-            .addAll(
-                main.getModules(
-                    CLUSTER_NAME,
-                    SERVERSET_PATH,
-                    zkClientConfig,
-                    STATS_URL_PREFIX,
-                    Optional.<String>absent()))
+            .addAll(main.getModules(CLUSTER_NAME, SERVERSET_PATH, zkClientConfig, STATS_URL_PREFIX))
             .add(new LifecycleModule())
             .add(new AppLauncherModule())
             .add(new ZooKeeperClientModule(zkClientConfig))

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/test/java/org/apache/aurora/scheduler/http/JettyServerModuleTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/JettyServerModuleTest.java b/src/test/java/org/apache/aurora/scheduler/http/JettyServerModuleTest.java
index c3e40d8..0d9aeff 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/JettyServerModuleTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/JettyServerModuleTest.java
@@ -13,13 +13,11 @@
  */
 package org.apache.aurora.scheduler.http;
 
-import java.net.InetSocketAddress;
-
 import javax.servlet.ServletContextListener;
 import javax.ws.rs.core.MediaType;
 
-import com.google.common.base.Optional;
 import com.google.common.base.Throwables;
+import com.google.common.net.HostAndPort;
 import com.google.common.testing.TearDown;
 import com.google.common.util.concurrent.RateLimiter;
 import com.google.inject.AbstractModule;
@@ -37,7 +35,6 @@ import com.sun.jersey.api.json.JSONConfiguration;
 import com.twitter.common.application.ShutdownRegistry.ShutdownRegistryImpl;
 import com.twitter.common.application.StartupRegistry;
 import com.twitter.common.application.modules.LifecycleModule;
-import com.twitter.common.application.modules.LocalServiceRegistry;
 import com.twitter.common.application.modules.StatsModule;
 import com.twitter.common.base.Command;
 import com.twitter.common.net.pool.DynamicHostSet;
@@ -50,7 +47,6 @@ import com.twitter.thrift.ServiceInstance;
 
 import org.apache.aurora.gen.ServerInfo;
 import org.apache.aurora.scheduler.SchedulerServicesModule;
-import org.apache.aurora.scheduler.app.LocalServiceRegistryWithOverrides;
 import org.apache.aurora.scheduler.async.OfferManager;
 import org.apache.aurora.scheduler.async.RescheduleCalculator;
 import org.apache.aurora.scheduler.async.TaskGroups.TaskGroupsSettings;
@@ -78,7 +74,7 @@ import static org.junit.Assert.assertNotNull;
 public abstract class JettyServerModuleTest extends EasyMockTest {
   private Injector injector;
   protected StorageTestUtil storage;
-  protected InetSocketAddress httpServer;
+  protected HostAndPort httpServer;
   protected Capture<HostChangeMonitor<ServiceInstance>> schedulerWatcher;
 
   /**
@@ -119,8 +115,6 @@ public abstract class JettyServerModuleTest extends EasyMockTest {
                     Amount.of(1L, Time.MILLISECONDS),
                     bindMock(BackoffStrategy.class),
                     RateLimiter.create(1000)));
-            bind(LocalServiceRegistryWithOverrides.Settings.class).toInstance(
-                new LocalServiceRegistryWithOverrides.Settings(Optional.<String>absent()));
             bind(new TypeLiteral<DynamicHostSet<ServiceInstance>>() { }).toInstance(schedulers);
             bindMock(CronJobManager.class);
             bindMock(LockManager.class);
@@ -153,16 +147,16 @@ public abstract class JettyServerModuleTest extends EasyMockTest {
       }
     });
     try {
+
       injector.getInstance(StartupRegistry.class).execute();
     } catch (Exception e) {
       throw Throwables.propagate(e);
     }
-    LocalServiceRegistry serviceRegistry = injector.getInstance(LocalServiceRegistry.class);
-    httpServer = serviceRegistry.getAuxiliarySockets().get("http");
+    httpServer = injector.getInstance(HttpService.class).getAddress();
   }
 
   protected String makeUrl(String path) {
-    return String.format("http://%s:%s%s", httpServer.getHostName(), httpServer.getPort(), path);
+    return String.format("http://%s:%s%s", httpServer.getHostText(), httpServer.getPort(), path);
   }
 
   protected WebResource.Builder getRequestBuilder(String path) {

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/test/java/org/apache/aurora/scheduler/http/LeaderRedirectTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/LeaderRedirectTest.java b/src/test/java/org/apache/aurora/scheduler/http/LeaderRedirectTest.java
index 7f80757..703b026 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/LeaderRedirectTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/LeaderRedirectTest.java
@@ -14,7 +14,6 @@
 package org.apache.aurora.scheduler.http;
 
 import java.util.Arrays;
-import java.util.Set;
 
 import javax.servlet.http.HttpServletRequest;
 
@@ -24,12 +23,6 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.net.HostAndPort;
-import com.google.inject.util.Providers;
-import com.twitter.common.application.ShutdownRegistry.ShutdownRegistryImpl;
-import com.twitter.common.application.modules.LifecycleModule.ServiceRunner;
-import com.twitter.common.application.modules.LocalServiceRegistry;
-import com.twitter.common.application.modules.LocalServiceRegistry.LocalService;
-import com.twitter.common.base.Commands;
 import com.twitter.common.net.pool.DynamicHostSet;
 import com.twitter.common.net.pool.DynamicHostSet.HostChangeMonitor;
 import com.twitter.common.net.pool.DynamicHostSet.MonitorException;
@@ -37,7 +30,6 @@ import com.twitter.common.testing.easymock.EasyMockTest;
 import com.twitter.thrift.Endpoint;
 import com.twitter.thrift.ServiceInstance;
 
-import org.apache.aurora.scheduler.app.LocalServiceRegistryWithOverrides;
 import org.easymock.Capture;
 import org.junit.Before;
 import org.junit.Ignore;
@@ -71,20 +63,10 @@ public class LeaderRedirectTest extends EasyMockTest {
     DynamicHostSet<ServiceInstance> schedulers =
         createMock(new Clazz<DynamicHostSet<ServiceInstance>>() { });
 
-    ServiceRunner fakeRunner = new ServiceRunner() {
-      @Override
-      public LocalService launch() {
-        return LocalService.auxiliaryService(HTTP_PORT_NAME, HTTP_PORT, Commands.NOOP);
-      }
-    };
-
-    Set<ServiceRunner> services = ImmutableSet.of(fakeRunner);
-    LocalServiceRegistry serviceRegistry =
-        new LocalServiceRegistry(Providers.of(services), new ShutdownRegistryImpl());
-    LocalServiceRegistryWithOverrides withOverrides =
-        new LocalServiceRegistryWithOverrides(serviceRegistry,
-            new LocalServiceRegistryWithOverrides.Settings(Optional.<String>absent()));
-    leaderRedirector = new LeaderRedirect(withOverrides, schedulers);
+    HttpService http = createMock(HttpService.class);
+    expect(http.getAddress()).andStubReturn(HostAndPort.fromParts("localhost", HTTP_PORT));
+
+    leaderRedirector = new LeaderRedirect(http, schedulers);
 
     monitorCapture = new Capture<>();
     expect(schedulers.watch(capture(monitorCapture))).andReturn(null);

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java b/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java
index c5c5f78..f8b134f 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/ServletFilterTest.java
@@ -108,7 +108,7 @@ public class ServletFilterTest extends JettyServerModuleTest {
     leaderRedirectSmokeTest(Status.OK);
 
     // This process is leading
-    setLeadingScheduler(httpServer.getHostName(), httpServer.getPort());
+    setLeadingScheduler(httpServer.getHostText(), httpServer.getPort());
     leaderRedirectSmokeTest(Status.OK);
 
     setLeadingScheduler("otherHost", 1234);

http://git-wip-us.apache.org/repos/asf/aurora/blob/165c4dff/src/test/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityIT.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityIT.java
index 45a23fd..6743d06 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/ApiSecurityIT.java
@@ -151,7 +151,7 @@ public class ApiSecurityIT extends JettyServerModuleTest {
 
   private AuroraAdmin.Client getClient(HttpClient httpClient) throws TTransportException {
     final TTransport httpClientTransport = new THttpClient(
-        "http://" + httpServer.getHostName() + ":" + httpServer.getPort() + API_PATH,
+        "http://" + httpServer.getHostText() + ":" + httpServer.getPort() + API_PATH,
         httpClient);
     addTearDown(new TearDown() {
       @Override