You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by st...@apache.org on 2014/11/25 20:21:42 UTC

[7/7] incubator-slider git commit: SLIDER-319 hooking up codahale metrics to source and in to web

SLIDER-319 hooking up codahale metrics to source and in to web


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/867b67d4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/867b67d4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/867b67d4

Branch: refs/heads/feature/SLIDER-319_codahale_metrics
Commit: 867b67d4e1b80ad1072a9347b310dc1f3d1d576b
Parents: f0a30fd
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 21 19:05:53 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Nov 25 19:20:47 2014 +0000

----------------------------------------------------------------------
 pom.xml                                         |  12 ++
 slider-core/pom.xml                             |  10 ++
 .../core/registry/retrieve/AMWebClient.java     | 129 +++++++++++++++++++
 .../registry/retrieve/RegistryRetriever.java    |  87 +------------
 .../server/appmaster/SliderAppMaster.java       |  35 ++---
 .../server/appmaster/monkey/ChaosEntry.java     |   2 -
 .../slider/server/appmaster/state/AppState.java |  11 +-
 .../server/appmaster/web/SliderAMWebApp.java    |  21 ++-
 .../slider/server/appmaster/web/WebAppApi.java  |  15 ++-
 .../server/appmaster/web/WebAppApiImpl.java     |  25 ++--
 .../server/appmaster/web/rest/RestPaths.java    |   5 +
 .../services/utility/MetricsBindingService.java | 128 ++++++++++++++++++
 .../standalone/TestStandaloneAgentAM.groovy     |  14 +-
 .../standalone/TestStandaloneAgentWeb.groovy    |  80 ++++++++++++
 .../appmaster/model/mock/MockAppState.groovy    |   5 +-
 .../view/TestClusterSpecificationBlock.groovy   |   2 +-
 .../web/view/TestContainerStatsBlock.groovy     |   2 +-
 .../appmaster/web/view/TestIndexBlock.groovy    |   2 +-
 .../web/rest/agent/TestAMAgentWebServices.java  |   5 +-
 .../management/TestAMManagementWebServices.java |   5 +-
 20 files changed, 450 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7fb955b..03a1907 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1224,6 +1224,18 @@
         <version>${metrics.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>com.codahale.metrics</groupId>
+        <artifactId>metrics-ganglia</artifactId>
+        <version>${metrics.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>com.codahale.metrics</groupId>
+        <artifactId>metrics-servlets</artifactId>
+        <version>${metrics.version}</version>
+      </dependency>
+
       <!-- ======================================================== -->
       <!-- Mocking -->
       <!-- ======================================================== -->

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/slider-core/pom.xml b/slider-core/pom.xml
index 2c3aaa6..6bb7c09 100644
--- a/slider-core/pom.xml
+++ b/slider-core/pom.xml
@@ -333,6 +333,16 @@
       <groupId>com.codahale.metrics</groupId>
       <artifactId>metrics-core</artifactId>
     </dependency>
+
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-ganglia</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-servlets</artifactId>
+    </dependency>
     
     <!-- ======================================================== -->
     <!-- service registry -->

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java b/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java
new file mode 100644
index 0000000..24b1525
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java
@@ -0,0 +1,129 @@
+/*
+ * 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.slider.core.registry.retrieve;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+
+/**
+ * Class to retrieve artifacts from the AM's web site. This sets up
+ * the redirection and security logic properly
+ */
+public class AMWebClient {
+  private static final Client client;
+  private static final Logger
+      log = LoggerFactory.getLogger(AMWebClient.class);
+
+
+  static {
+    ClientConfig clientConfig = new DefaultClientConfig();
+    clientConfig.getFeatures().put(
+        JSONConfiguration.FEATURE_POJO_MAPPING,
+        Boolean.TRUE);
+    clientConfig.getProperties().put(
+        URLConnectionClientHandler.PROPERTY_HTTP_URL_CONNECTION_SET_METHOD_WORKAROUND,
+        true);
+    URLConnectionClientHandler handler = getUrlConnectionClientHandler();
+    client = new Client(handler, clientConfig);
+    client.setFollowRedirects(true);
+  }
+
+  /**
+   * Get the Jersey Client
+   * @return
+   */
+  public static Client getClient() {
+    return client;
+  }
+
+  private static URLConnectionClientHandler getUrlConnectionClientHandler() {
+    return new URLConnectionClientHandler(new HttpURLConnectionFactory() {
+      @Override
+      public HttpURLConnection getHttpURLConnection(URL url)
+          throws IOException {
+        HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+        if (connection.getResponseCode() == HttpURLConnection.HTTP_MOVED_TEMP) {
+          // is a redirect - are we changing schemes?
+          String redirectLocation = connection.getHeaderField(HttpHeaders.LOCATION);
+          String originalScheme = url.getProtocol();
+          String redirectScheme = URI.create(redirectLocation).getScheme();
+          if (!originalScheme.equals(redirectScheme)) {
+            // need to fake it out by doing redirect ourselves
+            log.info("Protocol change during redirect. Redirecting {} to URL {}",
+                     url, redirectLocation);
+            URL redirectURL = new URL(redirectLocation);
+            connection = (HttpURLConnection) redirectURL.openConnection();
+          }
+        }
+        if (connection instanceof HttpsURLConnection) {
+          log.debug("Attempting to configure HTTPS connection using client "
+                    + "configuration");
+          final SSLFactory factory;
+          final SSLSocketFactory sf;
+          final HostnameVerifier hv;
+
+          try {
+            HttpsURLConnection c = (HttpsURLConnection) connection;
+            factory = new SSLFactory(SSLFactory.Mode.CLIENT, new Configuration());
+            factory.init();
+            sf = factory.createSSLSocketFactory();
+            hv = factory.getHostnameVerifier();
+            c.setSSLSocketFactory(sf);
+            c.setHostnameVerifier(hv);
+          } catch (Exception e) {
+            log.info("Unable to configure HTTPS connection from "
+                     + "configuration.  Leveraging JDK properties.");
+          }
+
+        }
+        return connection;
+      }
+    });
+  }
+
+  public WebResource resource(String url) {
+    WebResource resource = client.resource(url);
+    return resource;
+  }
+
+  public WebResource jsonResource(String url) {
+    WebResource resource = resource(url);
+    resource.type(MediaType.APPLICATION_JSON);
+    return resource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java b/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
index 1194270..0d32807 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
@@ -19,20 +19,12 @@
 package org.apache.slider.core.registry.retrieve;
 
 import com.beust.jcommander.Strings;
-import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.api.json.JSONConfiguration;
-import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
 import org.apache.hadoop.registry.client.exceptions.RegistryIOException;
 import org.apache.hadoop.registry.client.types.Endpoint;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.exceptions.ExceptionConverter;
 import org.apache.slider.core.registry.docstore.PublishedConfigSet;
@@ -43,17 +35,8 @@ import org.apache.slider.core.registry.info.CustomRegistryConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.SSLSocketFactory;
-import javax.ws.rs.core.HttpHeaders;
-import javax.ws.rs.core.MediaType;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.net.HttpURLConnection;
-import java.net.URI;
-import java.net.URL;
-import java.security.GeneralSecurityException;
 import java.util.List;
 
 /**
@@ -61,71 +44,13 @@ import java.util.List;
  * This hides the HTTP operations that take place to
  * get the actual content
  */
-public class RegistryRetriever {
+public class RegistryRetriever extends AMWebClient {
   private static final Logger log = LoggerFactory.getLogger(RegistryRetriever.class);
 
   private final String externalConfigurationURL;
   private final String internalConfigurationURL;
   private final String externalExportsURL;
   private final String internalExportsURL;
-  private static final Client jerseyClient;
-  
-  static {
-    ClientConfig clientConfig = new DefaultClientConfig();
-    clientConfig.getFeatures().put(
-        JSONConfiguration.FEATURE_POJO_MAPPING,
-        Boolean.TRUE);
-    clientConfig.getProperties().put(
-        URLConnectionClientHandler.PROPERTY_HTTP_URL_CONNECTION_SET_METHOD_WORKAROUND, true);
-    URLConnectionClientHandler handler = getUrlConnectionClientHandler();
-    jerseyClient = new Client(handler, clientConfig);
-    jerseyClient.setFollowRedirects(true);
-  }
-
-  private static URLConnectionClientHandler getUrlConnectionClientHandler() {
-    return new URLConnectionClientHandler(new HttpURLConnectionFactory() {
-      @Override
-      public HttpURLConnection getHttpURLConnection(URL url)
-          throws IOException {
-        HttpURLConnection connection = (HttpURLConnection) url.openConnection();
-        if (connection.getResponseCode() == HttpURLConnection.HTTP_MOVED_TEMP) {
-          // is a redirect - are we changing schemes?
-          String redirectLocation = connection.getHeaderField(HttpHeaders.LOCATION);
-          String originalScheme = url.getProtocol();
-          String redirectScheme = URI.create(redirectLocation).getScheme();
-          if (!originalScheme.equals(redirectScheme)) {
-            // need to fake it out by doing redirect ourselves
-            log.info("Protocol change during redirect. Redirecting {} to URL {}",
-                     url, redirectLocation);
-            URL redirectURL = new URL(redirectLocation);
-            connection = (HttpURLConnection) redirectURL.openConnection();
-          }
-        }
-        if (connection instanceof HttpsURLConnection) {
-          log.debug("Attempting to configure HTTPS connection using client "
-                    + "configuration");
-          final SSLFactory factory;
-          final SSLSocketFactory sf;
-          final HostnameVerifier hv;
-
-          try {
-            HttpsURLConnection c = (HttpsURLConnection) connection;
-            factory = new SSLFactory(SSLFactory.Mode.CLIENT, new Configuration());
-            factory.init();
-            sf = factory.createSSLSocketFactory();
-            hv = factory.getHostnameVerifier();
-            c.setSSLSocketFactory(sf);
-            c.setHostnameVerifier(hv);
-          } catch (Exception e) {
-            log.info("Unable to configure HTTPS connection from "
-                     + "configuration.  Leveraging JDK properties.");
-          }
-
-        }
-        return connection;
-      }
-    });
-  }
 
   public RegistryRetriever(String externalConfigurationURL, String internalConfigurationURL,
                            String externalExportsURL, String internalExportsURL) {
@@ -255,16 +180,6 @@ public class RegistryRetriever {
     }
   }
 
-  private WebResource resource(String url) {
-    WebResource resource = jerseyClient.resource(url);
-    return resource;
-  }
-
-  private WebResource jsonResource(String url) {
-    WebResource resource = resource(url);
-    resource.type(MediaType.APPLICATION_JSON);
-    return resource;
-  }
 
   /**
    * Get a complete configuration, with all values

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index e26bc3f..9530bc0 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -149,6 +149,7 @@ import org.apache.slider.server.appmaster.web.rest.RestPaths;
 import org.apache.slider.server.services.security.CertificateManager;
 import org.apache.slider.server.services.security.FsDelegationTokenManager;
 import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
+import org.apache.slider.server.services.utility.MetricsBindingService;
 import org.apache.slider.server.services.utility.WebAppService;
 import org.apache.slider.server.services.workflow.ServiceThreadFactory;
 import org.apache.slider.server.services.workflow.WorkflowExecutorService;
@@ -204,13 +205,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   public static final String SERVICE_CLASSNAME =
       "org.apache.slider.server.appmaster." + SERVICE_CLASSNAME_SHORT;
 
-
-  /**
-   * time to wait from shutdown signal being rx'd to telling
-   * the AM: {@value}
-   */
-  public static final int TERMINATION_SIGNAL_PROPAGATION_DELAY = 1000;
-
   public static final int HEARTBEAT_INTERVAL = 1000;
   public static final int NUM_RPC_HANDLERS = 5;
 
@@ -276,7 +270,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    * Ongoing state of the cluster: containers, nodes they
    * live on, etc.
    */
-  private final AppState appState = new AppState(new ProtobufRecordFactory());
+  private final AppState appState = new AppState(new ProtobufRecordFactory(),
+      metrics);
 
   private final ProviderAppState stateForProviders =
       new ProviderAppState("undefined", appState);
@@ -447,6 +442,9 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     addService(executorService);
 
     addService(actionQueues);
+    
+    addService(new MetricsBindingService("MetricsBindingService",
+        metrics));
     //init all child services
     super.serviceInit(conf);
   }
@@ -705,12 +703,15 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
       int port = getPortToRequest(instanceDefinition);
 
-      webApp = new SliderAMWebApp(registryOperations);
+      WebAppApi webAppApi = new WebAppApiImpl(this,
+          stateForProviders,
+          providerService,
+          certificateManager,
+          registryOperations,
+          metrics);
+      webApp = new SliderAMWebApp(webAppApi);
       WebApps.$for(SliderAMWebApp.BASE_PATH, WebAppApi.class,
-                   new WebAppApiImpl(this,
-                                     stateForProviders,
-                                     providerService,
-                                     certificateManager, registryOperations),
+                   webAppApi,
                    RestPaths.WS_CONTEXT)
                       .withHttpPolicy(serviceConf, HttpConfig.Policy.HTTP_ONLY)
                       .at(port)
@@ -1000,9 +1001,11 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     // Start up the agent web app and track the URL for it
     AgentWebApp agentWebApp = AgentWebApp.$for(AgentWebApp.BASE_PATH,
                      new WebAppApiImpl(this,
-                                       stateForProviders,
-                                       providerService,
-                                       certificateManager, registryOperations),
+                         stateForProviders,
+                         providerService,
+                         certificateManager,
+                         registryOperations,
+                         metrics),
                      RestPaths.AGENT_WS_CONTEXT)
         .withComponentConfig(getInstanceDefinition().getAppConfOperations()
                                  .getComponent(SliderKeys.COMPONENT_AM))

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
index 87a0aaa..a397e19 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
@@ -37,7 +37,6 @@ public class ChaosEntry {
   public final ChaosTarget target;
   public final long probability;
 
-  private final MetricRegistry metrics;
   private final Counter invocationCounter;
 
 
@@ -57,7 +56,6 @@ public class ChaosEntry {
     this.name = name;
     this.target = target;
     this.probability = probability;
-    this.metrics = metrics;
     invocationCounter =
         metrics.counter(MetricRegistry.name(ChaosEntry.class, name));
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
index 9956db2..348a063 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
@@ -18,6 +18,7 @@
 
 package org.apache.slider.server.appmaster.state;
 
+import com.codahale.metrics.MetricRegistry;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
@@ -261,8 +262,16 @@ public class AppState {
   
   private ContainerReleaseSelector containerReleaseSelector;
 
-  public AppState(AbstractRecordFactory recordFactory) {
+  private MetricRegistry metrics;
+
+  /**
+   * Create an instance
+   * @param recordFactory factory for YARN records
+   * @param metrics metrics registry or null if a new one 
+   */
+  public AppState(AbstractRecordFactory recordFactory, MetricRegistry metrics) {
     this.recordFactory = recordFactory;
+    this.metrics = metrics; 
   }
 
   public int getFailedCountainerCount() {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
index 7f30440..18c1a46 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
@@ -16,6 +16,10 @@
  */
 package org.apache.slider.server.appmaster.web;
 
+import com.codahale.metrics.servlets.HealthCheckServlet;
+import com.codahale.metrics.servlets.MetricsServlet;
+import com.codahale.metrics.servlets.PingServlet;
+import com.codahale.metrics.servlets.ThreadDumpServlet;
 import com.google.common.base.Preconditions;
 import com.sun.jersey.api.container.filter.GZIPContentEncodingFilter;
 import com.sun.jersey.api.core.ResourceConfig;
@@ -27,6 +31,7 @@ import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.slider.server.appmaster.web.rest.AMWadlGeneratorConfig;
 import org.apache.slider.server.appmaster.web.rest.AMWebServices;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
 import org.apache.slider.server.appmaster.web.rest.SliderJacksonJaxbJsonProvider;
 
 import java.util.HashMap;
@@ -42,11 +47,11 @@ public class SliderAMWebApp extends WebApp {
   public static final String CONTAINER_STATS = "/stats";
   public static final String CLUSTER_SPEC = "/spec";
 
-  private final RegistryOperations registry;
+  private final WebAppApi webAppApi;
 
-  public SliderAMWebApp(RegistryOperations registry) {
-    Preconditions.checkArgument(registry != null, "registry null");
-    this.registry = registry;
+  public SliderAMWebApp(WebAppApi webAppApi) {
+    Preconditions.checkArgument(webAppApi != null, "webAppApi null");
+    this.webAppApi = webAppApi;
   }
 
   @Override
@@ -76,8 +81,14 @@ public class SliderAMWebApp extends WebApp {
       serve(path).with(Dispatcher.class);
     }
 
+    // metrics
+//    serve(SYSTEM_HEALTHCHECK).with(new HealthCheckServlet());
+    serve(SYSTEM_METRICS).with(new MetricsServlet(webAppApi.getMetrics()));
+//    serve(SYSTEM_PING).with(PingServlet.class);
+//    serve(SYSTEM_THREADS).with(ThreadDumpServlet.class);
+
     String regex = "(?!/ws)";
-    serveRegex(regex).with(SliderDefaultWrapperServlet.class);
+    serveRegex(regex).with(SliderDefaultWrapperServlet.class); 
 
     Map<String, String> params = new HashMap<String, String>();
     params.put(ResourceConfig.FEATURE_IMPLICIT_VIEWABLES, "true");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
index c8c47c9..43f8cf6 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
@@ -16,6 +16,7 @@
  */
 package org.apache.slider.server.appmaster.web;
 
+import com.codahale.metrics.MetricRegistry;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.slider.api.SliderClusterProtocol;
 import org.apache.slider.providers.ProviderService;
@@ -35,35 +36,37 @@ public interface WebAppApi {
   /**
    * The {@link AppState} for the current cluster
    */
-  public StateAccessForProviders getAppState();
+  StateAccessForProviders getAppState();
   
   /**
    * The {@link ProviderService} for the current cluster
    */
-  public ProviderService getProviderService();
+  ProviderService getProviderService();
 
 
   /**
    * The {@link CertificateManager} for the current cluster
    */
-  public CertificateManager getCertificateManager();
+  CertificateManager getCertificateManager();
 
   /**
    * The {@link SliderClusterProtocol} for the current cluster
    */
-  public SliderClusterProtocol getClusterProtocol();
+  SliderClusterProtocol getClusterProtocol();
   
   /**
    * Generate a mapping from role name to its {@link RoleStatus}. Be aware that this
    * is a computed value and not just a getter
    */
-  public Map<String, RoleStatus> getRoleStatusByName();
+  Map<String, RoleStatus> getRoleStatusByName();
 
   /**
    * Returns an interface that can support the agent-based REST operations.
    */
-  public AgentRestOperations getAgentRestOperations();
+  AgentRestOperations getAgentRestOperations();
 
 
   RegistryOperations getRegistryOperations();
+
+  MetricRegistry getMetrics();
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
index 3b47ed1..81e6564 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
@@ -16,6 +16,7 @@
  */
 package org.apache.slider.server.appmaster.web;
 
+import com.codahale.metrics.MetricRegistry;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.slider.api.SliderClusterProtocol;
 import org.apache.slider.providers.ProviderService;
@@ -43,12 +44,14 @@ public class WebAppApiImpl implements WebAppApi {
   protected final ProviderService provider;
   protected final CertificateManager certificateManager;
   private final RegistryOperations registryOperations;
+  private final MetricRegistry metrics;
 
   public WebAppApiImpl(SliderClusterProtocol clusterProto,
       StateAccessForProviders appState,
       ProviderService provider,
       CertificateManager certificateManager,
-      RegistryOperations registryOperations) {
+      RegistryOperations registryOperations,
+      MetricRegistry metrics) {
     this.registryOperations = registryOperations;
     checkNotNull(clusterProto);
     checkNotNull(appState);
@@ -58,19 +61,14 @@ public class WebAppApiImpl implements WebAppApi {
     this.appState = appState;
     this.provider = provider;
     this.certificateManager = certificateManager;
+    this.metrics = metrics;
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.slider.server.appmaster.web.WebAppApi#getAppState()
-   */
   @Override
   public StateAccessForProviders getAppState() {
     return appState;
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.slider.server.appmaster.web.WebAppApi#getProviderService()
-   */
   @Override
   public ProviderService getProviderService() {
     return provider;
@@ -81,21 +79,15 @@ public class WebAppApiImpl implements WebAppApi {
     return certificateManager;
   }
 
-  /* (non-Javadoc)
-     * @see org.apache.slider.server.appmaster.web.WebAppApi#getClusterProtocol()
-     */
   @Override
   public SliderClusterProtocol getClusterProtocol() {
     return clusterProto;
   }
   
-  /* (non-Javadoc)
-   * @see org.apache.slider.server.appmaster.web.WebAppApi#getRoleStatusByName()
-   */
   @Override
   public Map<String,RoleStatus> getRoleStatusByName() {
     List<RoleStatus> roleStatuses = appState.cloneRoleStatusList();
-    TreeMap<String, RoleStatus> map =
+    Map<String, RoleStatus> map =
         new TreeMap<String, RoleStatus>();
     for (RoleStatus status : roleStatuses) {
       map.put(status.getName(), status);
@@ -112,4 +104,9 @@ public class WebAppApiImpl implements WebAppApi {
   public RegistryOperations getRegistryOperations() {
     return registryOperations;
   }
+
+  @Override
+  public MetricRegistry getMetrics() {
+    return metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
index 94f1e4c..75788f9 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
@@ -64,4 +64,9 @@ public class RestPaths {
   public static final String SLIDER_EXPORTS = "exports";
 
   public static final String SLIDER_CLASSPATH = "classpath";
+  public static final String SYSTEM = "/system";
+  public static final String SYSTEM_HEALTHCHECK = SYSTEM + "/healthcheck";
+  public static final String SYSTEM_METRICS = SYSTEM + "/metrics";
+  public static final String SYSTEM_PING = SYSTEM + "/ping";
+  public static final String SYSTEM_THREADS = SYSTEM + "/threads";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/main/java/org/apache/slider/server/services/utility/MetricsBindingService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/utility/MetricsBindingService.java b/slider-core/src/main/java/org/apache/slider/server/services/utility/MetricsBindingService.java
new file mode 100644
index 0000000..afd2c1a
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/services/utility/MetricsBindingService.java
@@ -0,0 +1,128 @@
+/*
+ * 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.slider.server.services.utility;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.ScheduledReporter;
+import com.codahale.metrics.ganglia.GangliaReporter;
+import info.ganglia.gmetric4j.gmetric.GMetric;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.service.AbstractService;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * YARN service which hooks up Codahale metrics to 
+ * Ganglia (if enabled)
+ */
+public class MetricsBindingService extends AbstractService {
+
+  /**
+   * {@value}
+   */
+  public static final String METRICS_GANGLIA_ENABLED =
+      "metrics.ganglia.enabled";
+  /**
+   * {@value}
+   */
+  public static final String METRICS_GANGLIA_HOST = "metrics.ganglia.host";
+
+  /**
+   * {@value}
+   */
+  public static final String METRICS_GANGLIA_PORT = "metrics.ganglia.port";
+
+  /**
+   * {@value}
+   */
+  public static final String METRICS_GANGLIA_VERSION_31 = "metrics.ganglia.version.31";
+
+  /**
+   * {@value}
+   */
+  public static final String METRICS_GANGLIA_REPORT_INTERVAL = "metrics.ganglia.report.interval";
+
+  /**
+   * {@value}
+   */
+  public static final int DEFAULT_GANGLIA_PORT = 8649;
+
+  private final MetricRegistry metrics;
+  private ScheduledReporter reporter;
+
+  public MetricsBindingService(String name,
+      MetricRegistry metrics) {
+    super(name);
+    this.metrics = metrics;
+  }
+
+  /**
+   * Instantiate...create a metric registry in the process
+   * @param name service name
+   */
+  public MetricsBindingService(String name) {
+    this(name, new MetricRegistry());
+  }
+
+  public MetricRegistry getMetrics() {
+    return metrics;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    Configuration conf = getConfig();
+    boolean enabled = conf.getBoolean(METRICS_GANGLIA_ENABLED, false);
+
+    if (enabled) {
+      String host = conf.getTrimmed(METRICS_GANGLIA_HOST, "");
+      int port = conf.getInt(METRICS_GANGLIA_PORT, DEFAULT_GANGLIA_PORT);
+      int interval = conf.getInt(METRICS_GANGLIA_REPORT_INTERVAL, 60);
+      int ttl = 1;
+      GMetric.UDPAddressingMode
+          mcast = GMetric.UDPAddressingMode.getModeForAddress(host);
+      boolean ganglia31 = conf.getBoolean(METRICS_GANGLIA_VERSION_31, true);
+
+      final GMetric ganglia =
+          new GMetric(
+              host, 
+              port,
+              mcast,
+              ttl,
+              ganglia31);
+      reporter = GangliaReporter.forRegistry(metrics)
+                                .convertRatesTo(TimeUnit.SECONDS)
+                                .convertDurationsTo(TimeUnit.MILLISECONDS)
+                                .build(ganglia);
+      reporter.start(interval, TimeUnit.SECONDS);
+    }
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    IOUtils.closeStream(reporter);
+    super.serviceStop();
+  }
+  
+  public boolean isEnabled() {
+    return reporter != null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
index 160f4b7..3707b85 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
@@ -44,7 +44,9 @@ import org.junit.Test
 @CompileStatic
 @Slf4j
 class TestStandaloneAgentAM  extends AgentMiniClusterTestBase {
-  
+
+  public static final String PORT_RANGE = "60000-60010"
+
   @After
   void fixclientname() {
     sliderClientClassName = DEFAULT_SLIDER_CLIENT
@@ -66,8 +68,8 @@ class TestStandaloneAgentAM  extends AgentMiniClusterTestBase {
 
     ApplicationReport report = waitForClusterLive(client)
     URI uri = new URI(report.originalTrackingUrl)
-    assert uri.port in [60000, 60001, 60002, 60003]
-    assert report.rpcPort in [60000, 60001, 60002, 60003]
+    assert uri.port in 60000..60010
+    assert report.rpcPort in 60000..60010
 
     logReport(report)
     List<ApplicationReport> apps = client.applications;
@@ -167,7 +169,7 @@ class TestStandaloneAgentAM  extends AgentMiniClusterTestBase {
 
     // do a quick registry listing here expecting a usage failure.
     ActionRegistryArgs registryArgs = new ActionRegistryArgs()
-    registryArgs.name=clustername;
+    registryArgs.name = clustername;
     def exitCode = client.actionRegistry(registryArgs)
     assert LauncherExitCodes.EXIT_USAGE == exitCode 
 
@@ -203,7 +205,7 @@ class TestStandaloneAgentAM  extends AgentMiniClusterTestBase {
     throws IOException, SliderException, LockAcquireFailedException {
       AggregateConf conf = builder.instanceDescription
       conf.appConfOperations.
-          globalOptions[SliderKeys.KEY_ALLOWED_PORT_RANGE]= "60000-60003"
+          globalOptions[SliderKeys.KEY_ALLOWED_PORT_RANGE]= PORT_RANGE
       super.persistInstanceDefinition(overwrite, appconfdir, builder)
     }
 
@@ -214,7 +216,7 @@ class TestStandaloneAgentAM  extends AgentMiniClusterTestBase {
                                           boolean debugAM)
     throws YarnException, IOException {
       instanceDefinition.appConfOperations.
-          globalOptions[SliderKeys.KEY_ALLOWED_PORT_RANGE] ="60000-60003"
+          globalOptions[SliderKeys.KEY_ALLOWED_PORT_RANGE] =PORT_RANGE
       return super.launchApplication(clustername, clusterDirectory, instanceDefinition, debugAM)
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
new file mode 100644
index 0000000..717d98f
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
@@ -0,0 +1,80 @@
+/*
+ * 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.slider.agent.standalone
+
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+import org.apache.hadoop.yarn.exceptions.YarnException
+import org.apache.slider.agent.AgentMiniClusterTestBase
+import org.apache.slider.api.ClusterNode
+import org.apache.slider.client.SliderClient
+import org.apache.slider.common.SliderKeys
+import org.apache.slider.common.params.ActionRegistryArgs
+import org.apache.slider.common.tools.Duration
+import org.apache.slider.core.build.InstanceBuilder
+import org.apache.slider.core.conf.AggregateConf
+import org.apache.slider.core.exceptions.SliderException
+import org.apache.slider.core.launch.LaunchedApplication
+import org.apache.slider.core.main.LauncherExitCodes
+import org.apache.slider.core.main.ServiceLauncher
+import org.apache.slider.core.persist.LockAcquireFailedException
+import org.apache.slider.core.registry.retrieve.AMWebClient
+import org.apache.slider.server.appmaster.web.rest.RestPaths
+import org.junit.After
+import org.junit.Test
+
+@CompileStatic
+@Slf4j
+class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
+
+  
+  @Test
+  public void testStandaloneAgentWeb() throws Throwable {
+
+    describe "create a standalone AM then perform actions on it"
+    //launch fake master
+    String clustername = createMiniCluster("", configuration, 1, true)
+
+
+    ServiceLauncher<SliderClient> launcher =
+        createStandaloneAM(clustername, true, false)
+    SliderClient client = launcher.service
+    addToTeardown(client);
+
+    ApplicationReport report = waitForClusterLive(client)
+    def realappmaster = report.originalTrackingUrl
+    GET(realappmaster)
+    def metrics = GET(realappmaster, RestPaths.SYSTEM_METRICS)
+    log.info metrics
+
+    def appmaster = report.trackingUrl
+
+//    GET(appmaster)
+
+//    GET(appmaster, RestPaths.SYSTEM_HEALTHCHECK)
+//    GET(appmaster, RestPaths.SYSTEM_PING)
+    
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy
index e683587..7abb123 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy
@@ -17,6 +17,7 @@
 
 package org.apache.slider.server.appmaster.model.mock
 
+import com.codahale.metrics.MetricRegistry
 import org.apache.slider.providers.ProviderRole
 import org.apache.slider.server.appmaster.state.AbstractRecordFactory
 import org.apache.slider.server.appmaster.state.AppState
@@ -27,7 +28,7 @@ import org.apache.slider.server.appmaster.state.AppState
 class MockAppState extends AppState {
 
   public MockAppState(AbstractRecordFactory recordFactory) {
-    super(recordFactory);
+    super(recordFactory, new MetricRegistry());
   }
 
   long time = 0;
@@ -36,7 +37,7 @@ class MockAppState extends AppState {
    * Instance with a mock record factory
    */
   public MockAppState() {
-    super(new MockRecordFactory());
+    super(new MockRecordFactory(), new MetricRegistry());
   }
 
   public Map<String, ProviderRole> getRoleMap() {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
index c49abec..164d93c 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
@@ -57,7 +57,7 @@ public class TestClusterSpecificationBlock {
         providerAppState,
         providerService,
         null,
-        null);
+        null, null);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
index 9292285..4f1d52d 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
@@ -64,7 +64,7 @@ public class TestContainerStatsBlock extends BaseMockAppStateTest {
         providerAppState,
         providerService,
         null,
-        null);
+        null, metricRegistry);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
index d232ecb..ff24b82 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
@@ -56,7 +56,7 @@ public class TestIndexBlock extends BaseMockAppStateTest {
         providerAppState,
         providerService,
         null,
-        null);
+        null, metricRegistry);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
index faec5d8..db8223a 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
@@ -18,6 +18,7 @@
 
 package org.apache.slider.server.appmaster.web.rest.agent;
 
+import com.codahale.metrics.MetricRegistry;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -121,7 +122,7 @@ public class TestAMAgentWebServices {
           historyPath =
           new org.apache.hadoop.fs.Path(historyWorkDir.toURI());
       fs.delete(historyPath, true);
-      appState = new AppState(new MockRecordFactory());
+      appState = new AppState(new MockRecordFactory(), new MetricRegistry());
       appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES);
       appState.buildInstance(
           factory.newInstanceDefinition(0, 0, 0),
@@ -138,7 +139,7 @@ public class TestAMAgentWebServices {
                                                              appState);
 
     slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
-                               new MockProviderService(), null, null);
+                               new MockProviderService(), null, null, null);
 
     MapOperations compOperations = new MapOperations();
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/867b67d4/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
index 20889ac..9bfcd25 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
@@ -18,6 +18,7 @@
 
 package org.apache.slider.server.appmaster.web.rest.management;
 
+import com.codahale.metrics.MetricRegistry;
 import com.google.inject.Guice;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
@@ -167,7 +168,7 @@ public class TestAMManagementWebServices extends JerseyTest {
               historyPath =
               new org.apache.hadoop.fs.Path(historyWorkDir.toURI());
           fs.delete(historyPath, true);
-          appState = new AppState(new MockRecordFactory());
+          appState = new AppState(new MockRecordFactory(), new MetricRegistry());
           appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES);
           appState.buildInstance(
               factory.newInstanceDefinition(0, 0, 0),
@@ -191,7 +192,7 @@ public class TestAMManagementWebServices extends JerseyTest {
             appState);
 
         slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
-                                   new MockProviderService(), null, null);
+                                   new MockProviderService(), null, null, null);
 
         bind(SliderJacksonJaxbJsonProvider.class);
         bind(MockSliderAMWebServices.class);