You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ka...@apache.org on 2014/12/18 01:34:14 UTC

hadoop git commit: YARN-2203. [YARN-1492] Web UI for cache manager. (Chris Trezzo via kasha)

Repository: hadoop
Updated Branches:
  refs/heads/trunk a1bd14096 -> b7f64823e


YARN-2203. [YARN-1492] Web UI for cache manager. (Chris Trezzo via kasha)


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

Branch: refs/heads/trunk
Commit: b7f64823e11f745783607ae5f3f97b5e8e64c389
Parents: a1bd140
Author: Karthik Kambatla <ka...@apache.org>
Authored: Wed Dec 17 16:32:21 2014 -0800
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Wed Dec 17 16:32:21 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  2 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  7 ++
 .../src/main/resources/yarn-default.xml         |  6 ++
 .../sharedcachemanager/SharedCacheManager.java  |  8 ++
 .../webapp/SCMController.java                   | 43 +++++++++
 .../webapp/SCMMetricsInfo.java                  | 70 +++++++++++++++
 .../webapp/SCMOverviewPage.java                 | 95 ++++++++++++++++++++
 .../sharedcachemanager/webapp/SCMWebServer.java | 91 +++++++++++++++++++
 8 files changed, 322 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7f64823/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 4c5fc77..7af08b9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -65,6 +65,8 @@ Release 2.7.0 - UNRELEASED
     YARN-2880. Added a test to make sure node labels will be recovered
     if RM restart is enabled. (Rohith Sharmaks via jianhe)
 
+    YARN-2203. [YARN-1492] Web UI for cache manager. (Chris Trezzo via kasha)
+
   IMPROVEMENTS
 
     YARN-2950. Change message to mandate, not suggest JS requirement on UI.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7f64823/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 55073c5..d0cf761 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -1413,6 +1413,13 @@ public class YarnConfiguration extends Configuration {
       SHARED_CACHE_PREFIX + "admin.thread-count";
   public static final int DEFAULT_SCM_ADMIN_CLIENT_THREAD_COUNT = 1;
 
+  /** The address of the SCM web application. */
+  public static final String SCM_WEBAPP_ADDRESS =
+      SHARED_CACHE_PREFIX + "webapp.address";
+  public static final int DEFAULT_SCM_WEBAPP_PORT = 8788;
+  public static final String DEFAULT_SCM_WEBAPP_ADDRESS =
+      "0.0.0.0:" + DEFAULT_SCM_WEBAPP_PORT;
+
   // In-memory SCM store configuration
   
   public static final String IN_MEMORY_STORE_PREFIX =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7f64823/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 73a6b5d..ff4bf2b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1445,6 +1445,12 @@
   </property>
 
   <property>
+    <description>The address of the web application in the SCM (shared cache manager)</description>
+    <name>yarn.sharedcache.webapp.address</name>
+    <value>0.0.0.0:8788</value>
+  </property>
+
+  <property>
     <description>The frequency at which a cleaner task runs.
     Specified in minutes.</description>
     <name>yarn.sharedcache.cleaner.period-mins</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7f64823/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
index d22fa51..331e29e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.sharedcachemanager.store.SCMStore;
+import org.apache.hadoop.yarn.server.sharedcachemanager.webapp.SCMWebServer;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -77,6 +78,9 @@ public class SharedCacheManager extends CompositeService {
     SCMAdminProtocolService saps = createSCMAdminProtocolService(cs);
     addService(saps);
 
+    SCMWebServer webUI = createSCMWebServer(this);
+    addService(webUI);
+
     // init metrics
     DefaultMetricsSystem.initialize("SharedCacheManager");
     JvmMetrics.initSingleton("SharedCacheManager", null);
@@ -121,6 +125,10 @@ public class SharedCacheManager extends CompositeService {
     return new SCMAdminProtocolService(cleanerService);
   }
 
+  private SCMWebServer createSCMWebServer(SharedCacheManager scm) {
+    return new SCMWebServer(scm);
+  }
+
   @Override
   protected void serviceStop() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7f64823/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMController.java
new file mode 100644
index 0000000..f597d14
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMController.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.sharedcachemanager.webapp;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.webapp.Controller;
+
+/**
+ * The controller class for the shared cache manager web app.
+ */
+@Private
+@Unstable
+public class SCMController extends Controller {
+  @Override
+  public void index() {
+    setTitle("Shared Cache Manager");
+  }
+
+  /**
+   * It is referenced in SCMWebServer.SCMWebApp.setup()
+   */
+  @SuppressWarnings("unused")
+  public void overview() {
+    render(SCMOverviewPage.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7f64823/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMMetricsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMMetricsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMMetricsInfo.java
new file mode 100644
index 0000000..24aa145
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMMetricsInfo.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.sharedcachemanager.webapp;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.CleanerMetrics;
+import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.ClientSCMMetrics;
+import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.SharedCacheUploaderMetrics;
+
+/**
+ * This class is used to summarize useful shared cache manager metrics for the
+ * webUI display.
+ */
+@XmlRootElement(name = "SCMMetrics")
+@XmlAccessorType(XmlAccessType.FIELD)
+@Private
+@Unstable
+public class SCMMetricsInfo {
+  protected long totalDeletedFiles;
+  protected long totalProcessedFiles;
+  protected long cacheHits;
+  protected long cacheMisses;
+  protected long cacheReleases;
+  protected long acceptedUploads;
+  protected long rejectedUploads;
+
+  public SCMMetricsInfo() {
+  }
+  
+  public SCMMetricsInfo(CleanerMetrics cleanerMetrics,
+      ClientSCMMetrics clientSCMMetrics,
+      SharedCacheUploaderMetrics scmUploaderMetrics) {
+    totalDeletedFiles = cleanerMetrics.getTotalDeletedFiles();
+    totalProcessedFiles = cleanerMetrics.getTotalProcessedFiles();
+    cacheHits = clientSCMMetrics.getCacheHits();
+    cacheMisses = clientSCMMetrics.getCacheMisses();
+    cacheReleases = clientSCMMetrics.getCacheReleases();
+    acceptedUploads = scmUploaderMetrics.getAcceptedUploads();
+    rejectedUploads = scmUploaderMetrics.getRejectUploads();
+  }
+
+  public long getTotalDeletedFiles() { return totalDeletedFiles; }
+  public long getTotalProcessedFiles() { return totalProcessedFiles; }
+  public long getCacheHits() { return cacheHits; }
+  public long getCacheMisses() { return cacheMisses; }
+  public long getCacheReleases() { return cacheReleases; }
+  public long getAcceptedUploads() { return acceptedUploads; }
+  public long getRejectUploads() { return rejectedUploads; }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7f64823/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java
new file mode 100644
index 0000000..27944d3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java
@@ -0,0 +1,95 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.sharedcachemanager.webapp;
+
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.sharedcachemanager.SharedCacheManager;
+import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.CleanerMetrics;
+import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.ClientSCMMetrics;
+import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.SharedCacheUploaderMetrics;
+import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.InfoBlock;
+import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout;
+
+import com.google.inject.Inject;
+
+/**
+ * This class is to render the shared cache manager web ui overview page.
+ */
+@Private
+@Unstable
+public class SCMOverviewPage extends TwoColumnLayout {
+
+  @Override protected void preHead(Page.HTML<_> html) {
+    set(ACCORDION_ID, "nav");
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
+  }
+
+  @Override protected Class<? extends SubView> content() {
+    return SCMOverviewBlock.class;
+  }
+
+  @Override
+  protected Class<? extends SubView> nav() {
+    return SCMOverviewNavBlock.class;
+  }
+
+  static private class SCMOverviewNavBlock extends HtmlBlock {
+    @Override
+    protected void render(Block html) {
+      html.div("#nav").h3("Tools").ul().li().a("/conf", "Configuration")._()
+          .li().a("/stacks", "Thread dump")._().li().a("/logs", "Logs")._()
+          .li().a("/metrics", "Metrics")._()._()._();
+    }
+  }
+
+  static private class SCMOverviewBlock extends HtmlBlock {
+    final SharedCacheManager scm;
+
+    @Inject
+    SCMOverviewBlock(SharedCacheManager scm, ViewContext ctx) {
+      super(ctx);
+      this.scm = scm;
+    }
+
+    @Override
+    protected void render(Block html) {
+      SCMMetricsInfo metricsInfo = new SCMMetricsInfo(
+          CleanerMetrics.getInstance(), ClientSCMMetrics.getInstance(),
+              SharedCacheUploaderMetrics.getInstance());
+      info("Shared Cache Manager overview").
+          _("Started on:", Times.format(scm.getStartTime())).
+          _("Cache hits: ", metricsInfo.getCacheHits()).
+          _("Cache misses: ", metricsInfo.getCacheMisses()).
+          _("Cache releases: ", metricsInfo.getCacheReleases()).
+          _("Accepted uploads: ", metricsInfo.getAcceptedUploads()).
+          _("Rejected uploads: ", metricsInfo.getRejectUploads()).
+          _("Deleted files by the cleaner: ", metricsInfo.getTotalDeletedFiles()).
+          _("Processed files by the cleaner: ", metricsInfo.getTotalProcessedFiles());
+      html._(InfoBlock.class);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7f64823/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java
new file mode 100644
index 0000000..b81ed29
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.sharedcachemanager.webapp;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.sharedcachemanager.SharedCacheManager;
+import org.apache.hadoop.yarn.webapp.WebApp;
+import org.apache.hadoop.yarn.webapp.WebApps;
+
+/**
+ * A very simple web interface for the metrics reported by
+ * {@link org.apache.hadoop.yarn.server.sharedcachemanager.SharedCacheManager}
+ * TODO: Security for web ui (See YARN-2774)
+ */
+@Private
+@Unstable
+public class SCMWebServer extends AbstractService {
+  private static final Log LOG = LogFactory.getLog(SCMWebServer.class);
+
+  private final SharedCacheManager scm;
+  private WebApp webApp;
+  private String bindAddress;
+
+  public SCMWebServer(SharedCacheManager scm) {
+    super(SCMWebServer.class.getName());
+    this.scm = scm;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    this.bindAddress = getBindAddress(conf);
+    super.serviceInit(conf);
+  }
+
+  private String getBindAddress(Configuration conf) {
+    return conf.get(YarnConfiguration.SCM_WEBAPP_ADDRESS,
+        YarnConfiguration.DEFAULT_SCM_WEBAPP_ADDRESS);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    SCMWebApp scmWebApp = new SCMWebApp(scm);
+    this.webApp = WebApps.$for("sharedcache").at(bindAddress).start(scmWebApp);
+    LOG.info("Instantiated " + SCMWebApp.class.getName() + " at " + bindAddress);
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (this.webApp != null) {
+      this.webApp.stop();
+    }
+  }
+
+  private class SCMWebApp extends WebApp {
+    private final SharedCacheManager scm;
+
+    public SCMWebApp(SharedCacheManager scm) {
+      this.scm = scm;
+    }
+
+    @Override
+    public void setup() {
+      if (scm != null) {
+        bind(SharedCacheManager.class).toInstance(scm);
+      }
+      route("/", SCMController.class, "overview");
+    }
+  }
+}
\ No newline at end of file