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/12/18 22:33:28 UTC

[14/26] incubator-slider git commit: SLIDER-711 Implement initial read-only view of REST resources. Core structure and live/resources.json

SLIDER-711 Implement initial read-only view of REST resources. Core structure and live/resources.json


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

Branch: refs/heads/develop
Commit: fad1b9784bc0576b6cdc187f29fb4ba7c0bde289
Parents: 5b81968
Author: Steve Loughran <st...@apache.org>
Authored: Tue Dec 16 13:31:06 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Dec 16 20:25:40 2014 +0000

----------------------------------------------------------------------
 .../org/apache/slider/api/ResourceKeys.java     |   2 +-
 .../java/org/apache/slider/api/StatusKeys.java  |  41 ++++++-
 .../slider/core/conf/ConfTreeOperations.java    |   3 +-
 .../server/appmaster/web/SliderAMWebApp.java    |   1 -
 .../appmaster/web/rest/AMWebServices.java       |  24 ++++-
 .../web/rest/AbstractSliderResource.java        |  31 +++++-
 .../server/appmaster/web/rest/RestPaths.java    |  19 ++++
 .../rest/application/ApplicationResource.java   | 108 +++++++++++++++++++
 .../web/rest/application/package-info.java      |  24 +++++
 .../application/resources/CachedContent.java    |  92 ++++++++++++++++
 .../application/resources/ContentCache.java     |  26 +++++
 .../resources/LiveResourcesRefresher.java       |  69 ++++++++++++
 .../resources/ResourceRefresher.java            |  31 ++++++
 .../web/rest/management/ManagementResource.java |  46 +-------
 .../web/rest/publisher/PublisherResource.java   |   4 +-
 .../TestMockContainerResourceAllocations.groovy |   8 +-
 16 files changed, 468 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java b/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
index f3eb963..52633f4 100644
--- a/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
@@ -44,7 +44,7 @@ public interface ResourceKeys {
 
 
   /**
-   * #of instances of a component
+   * #of instances of a component: {@value}
    *
   */
   String COMPONENT_INSTANCES = "yarn.component.instances";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java b/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
index 4e46605..92cc8aa 100644
--- a/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
@@ -17,7 +17,7 @@
  */
 
 package org.apache.slider.api;
-
+import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES;
 /**
  * Contains status and statistics keys
  */
@@ -71,4 +71,43 @@ public interface StatusKeys {
   String INFO_AM_AGENT_OPS_PORT = "info.am.agent.ops.port";
   String INFO_AM_AGENT_OPS_URL = "info.am.agent.ops.url";
   String INFO_AM_AGENT_STATUS_URL = "info.am.agent.status.url";
+
+      /**
+       * info: #of instances of a component requested: {@value}
+       *
+       */
+  String COMPONENT_INSTANCES_ACTUAL = COMPONENT_INSTANCES + ".actual";
+
+  /**
+   * info: #of instances of a component requested: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_REQUESTED = COMPONENT_INSTANCES + ".requested";
+
+  /**
+   * info: #of instances of a component being released: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_RELEASING = COMPONENT_INSTANCES + ".releasing";
+
+  /**
+   * info: #of instances of a component failed: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_FAILED = COMPONENT_INSTANCES + ".failed";
+
+  /**
+   * info: #of instances of a component started: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_STARTED = COMPONENT_INSTANCES + ".started";
+
+
+  /**
+   * info: #of instances of a component completed: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_COMPLETED = COMPONENT_INSTANCES + ".completed";
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java b/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
index e946e43..dca58aa 100644
--- a/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
+++ b/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
@@ -429,9 +429,8 @@ public class ConfTreeOperations {
    * @param option option name
    * @param val integer value
    */
-  public void setRoleOpt(String role, String option, int val) {
+  public void setComponentOpt(String role, String option, int val) {
     setComponentOpt(role, option, Integer.toString(val));
   }
 
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/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 f2fc903..84f0eba 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,7 +16,6 @@
  */
 package org.apache.slider.server.appmaster.web;
 
-import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.servlets.HealthCheckServlet;
 import com.codahale.metrics.servlets.MetricsServlet;
 import com.codahale.metrics.servlets.PingServlet;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
index 30db98e..03bf703 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
@@ -20,39 +20,53 @@ import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentResource;
+import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource;
 import org.apache.slider.server.appmaster.web.rest.management.ManagementResource;
 import org.apache.slider.server.appmaster.web.rest.publisher.PublisherResource;
 import org.apache.slider.server.appmaster.web.rest.registry.RegistryResource;
 
 import javax.ws.rs.Path;
 
-/** The available REST services exposed by a slider AM. */
+/**
+ *  The available REST services exposed by a slider AM. 
+ */
 @Singleton
 @Path(RestPaths.SLIDER_CONTEXT_ROOT)
 public class AMWebServices {
   
   /** AM/WebApp info object */
   private WebAppApi slider;
+  private final ManagementResource managementResource;
+  private final PublisherResource publisherResource;
+  private final RegistryResource registryResource;
+  private final ApplicationResource applicationResource;
 
   @Inject
   public AMWebServices(WebAppApi slider) {
     this.slider = slider;
+    managementResource = new ManagementResource(slider);
+    publisherResource = new PublisherResource(slider);
+    registryResource = new RegistryResource(slider);
+    applicationResource = new ApplicationResource(slider);
   }
 
   @Path(RestPaths.SLIDER_SUBPATH_MANAGEMENT)
   public ManagementResource getManagementResource() {
-    return new ManagementResource(slider);
+    return managementResource;
   }
 
   @Path(RestPaths.SLIDER_SUBPATH_PUBLISHER)
   public PublisherResource getPublisherResource() {
-    return new PublisherResource(slider);
+    return publisherResource;
   }
  
   @Path(RestPaths.SLIDER_SUBPATH_REGISTRY)
   public RegistryResource getRegistryResource() {
-    return new RegistryResource(slider);
+    return registryResource;
   }
   
-  
+  @Path(RestPaths.SLIDER_SUBPATH_APPLICATION)
+  public ApplicationResource getApplicationResource() {
+    return applicationResource;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
index 4f41815..90fe6fe 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
@@ -19,16 +19,45 @@
 package org.apache.slider.server.appmaster.web.rest;
 
 import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response;
+import java.net.URI;
+import java.net.URL;
 
 /**
  * Abstract resource base class for REST resources
  * that use the slider WebAppApi
  */
 public abstract class AbstractSliderResource {
-
+  private static final Logger log =
+      LoggerFactory.getLogger(AbstractSliderResource.class);
   protected final WebAppApi slider;
 
   public AbstractSliderResource(WebAppApi slider) {
     this.slider = slider;
   }
+
+
+  /**
+   * Generate a redirect to the WASL
+   * @param request to base the URL on
+   * @return a 302 response
+   */
+  protected Response redirectToAppWadl(HttpServletRequest request) {
+    try {
+      URI location = new URL(request.getScheme(),
+          request.getServerName(),
+          request.getServerPort(),
+          RestPaths.APPLICATION_WADL).toURI();
+      return Response.temporaryRedirect(location).build();
+    } catch (Exception e) {
+      log.error("Error during redirect to WADL", e);
+      throw new WebApplicationException(Response.serverError().build());
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/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 0df617f..a264f77 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
@@ -61,6 +61,9 @@ public class RestPaths {
   public static final String SLIDER_PATH_PUBLISHER = SLIDER_CONTEXT_ROOT
                                       + SLIDER_SUBPATH_PUBLISHER;
 
+  /**
+   * Registry subpath: {@value} 
+   */
   public static final String SLIDER_SUBPATH_REGISTRY = "/registry";
 
   /**
@@ -112,4 +115,20 @@ public class RestPaths {
    * Codahale Metrics - thread dump: {@value}
    */
   public static final String SYSTEM_THREADS = SYSTEM + "/threads";
+
+
+  /**
+   * application subpath
+   */
+  public static final String SLIDER_SUBPATH_APPLICATION = "/application";
+
+
+  /**
+   * management path: {@value}
+   */
+  public static final String SLIDER_PATH_APPLICATION =
+      SLIDER_CONTEXT_ROOT + SLIDER_SUBPATH_APPLICATION;
+
+
+  public static final String APPLICATION_WADL = "/application.wadl";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
new file mode 100644
index 0000000..2d63269
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
@@ -0,0 +1,108 @@
+/*
+ * 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.appmaster.web.rest.application;
+
+import com.google.common.collect.Lists;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
+import org.apache.slider.server.appmaster.web.rest.application.resources.CachedContent;
+import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
+import org.apache.slider.server.appmaster.web.rest.application.resources.LiveResourcesRefresher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import java.util.List;
+
+public class ApplicationResource extends AbstractSliderResource {
+  private static final Logger log =
+      LoggerFactory.getLogger(ApplicationResource.class);
+  public static final String LIVE_RESOURCES_JSON = "/live/resources.json";
+
+  ContentCache cache = new ContentCache();
+  public static final int LIFESPAN = 1000;
+
+  public ApplicationResource(WebAppApi slider) {
+    super(slider);
+    cache.put(LIVE_RESOURCES_JSON,
+        new CachedContent<ConfTree>(LIFESPAN,
+            new LiveResourcesRefresher(slider.getAppState())));
+  }
+
+  /**
+   * Build a new JSON-marshallable list of string elements
+   * @param elements elements
+   * @return something that can be returned
+   */
+  private List<String> toJsonList(String... elements) {
+    return Lists.newArrayList(elements);
+  }
+
+  @GET
+  @Path("/")
+  @Produces({MediaType.APPLICATION_JSON})
+  public List<String> getRoot() {
+    return toJsonList("model", "live", "actions");
+  }
+
+  @GET
+  @Path("/model")
+  @Produces({MediaType.APPLICATION_JSON})
+  public List<String> getModel() {
+    return toJsonList("desired", "resolved");
+  }
+
+  @GET
+  @Path("/model/")
+  @Produces({MediaType.APPLICATION_JSON})
+  public List<String> getModelSlash() {
+    return getModel();
+  }
+
+  @GET
+  @Path("/live")
+  @Produces({MediaType.APPLICATION_JSON})
+  public List<String> getLive() {
+    return toJsonList("resources",
+        "containers",
+        "components",
+        "nodes",
+        "statistics",
+        "internal");
+  }
+
+  @GET
+  @Path("/live/")
+  @Produces({MediaType.APPLICATION_JSON})
+  public List<String> getLiveSlash() {
+    return getLive();
+  }
+
+  @GET
+  @Path(LIVE_RESOURCES_JSON)
+  @Produces({MediaType.APPLICATION_JSON})
+  public Object getLiveResources() {
+    return cache.get(LIVE_RESOURCES_JSON).get();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java
new file mode 100644
index 0000000..34eb82c
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+
+/**
+ * This package contains resources related to the application
+ * REST API
+ */
+package org.apache.slider.server.appmaster.web.rest.application;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
new file mode 100644
index 0000000..c2edd61
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
@@ -0,0 +1,92 @@
+/*
+ * 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.appmaster.web.rest.application.resources;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Time;
+
+/**
+ * A cached resource is one that can be stored and served up, with a refresh 
+ * only taking place when the expiry happens
+ * @param <T>
+ */
+public class CachedContent<T> {
+  
+  private T cachedValue;
+  private long expires;
+  private final long lifespan;
+  private final ResourceRefresher<T> refresh;
+
+  public CachedContent(long lifespan,
+      ResourceRefresher<T> refresh) {
+    this.lifespan = lifespan;
+    this.refresh = refresh;
+  }
+
+  /**
+   * Get the value, triggering a refresh if needed
+   * @return the cached or latest value.
+   */
+  public T get() {
+    maybeRefresh();
+    return cachedValue;
+  }
+
+  /**
+   * Get the cached value without any expiry check
+   * @return the last value set. May be null.
+   */
+  public T getCachedValue() {
+    return cachedValue;
+  }
+
+  /**
+   * Get the lifespan in millis of the cached value
+   * @return the lifespan
+   */
+  public long getLifespan() {
+    return lifespan;
+  }
+
+  /**
+   * Maybe refresh the content
+   * @return true if a refresh took place.
+   */
+  public boolean maybeRefresh() {
+    long now = Time.monotonicNow();
+    if (now > expires) {
+      forceRefresh();
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Force a refresh and reset the expiry counter
+   * @return the new value
+   */
+  public T forceRefresh() {
+    T updated = refresh.refresh();
+    Preconditions.checkNotNull(updated);
+    cachedValue = updated;
+    expires = Time.monotonicNow() + lifespan;
+    return updated;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
new file mode 100644
index 0000000..0cc9eb4
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
@@ -0,0 +1,26 @@
+/*
+ * 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.appmaster.web.rest.application.resources;
+
+import java.util.HashMap;
+
+public class ContentCache extends HashMap<String, CachedContent> {
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
new file mode 100644
index 0000000..5c46a64
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
@@ -0,0 +1,69 @@
+/*
+ * 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.appmaster.web.rest.application.resources;
+
+import org.apache.slider.api.StatusKeys;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.server.appmaster.state.RoleStatus;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+import java.util.Map;
+
+public class LiveResourcesRefresher implements ResourceRefresher<ConfTree> {
+
+  private final StateAccessForProviders state;
+
+  public LiveResourcesRefresher(StateAccessForProviders state) {
+    this.state = state;
+  }
+
+  @Override
+  public ConfTree refresh() {
+    ConfTree resources = new ConfTree();
+    ConfTreeOperations ops = new ConfTreeOperations(resources);
+    // snapshot resources
+    state.getResourcesSnapshot();
+    // then add actual values
+    Map<Integer, RoleStatus> roleStatusMap = state.getRoleStatusMap();
+    
+    for (RoleStatus status : roleStatusMap.values()) {
+      String name = status.getName();
+      ops.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_REQUESTED,
+          status.getRequested());
+      ops.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_ACTUAL,
+          status.getActual());
+      ops.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_RELEASING,
+          status.getReleasing());
+      ops.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_FAILED,
+          status.getFailed());
+      ops.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_COMPLETED,
+          status.getCompleted());
+      ops.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_STARTED,
+          status.getStarted());
+    }
+    return resources;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
new file mode 100644
index 0000000..9822fbc
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
@@ -0,0 +1,31 @@
+/*
+ * 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.appmaster.web.rest.application.resources;
+
+/**
+ * Interface which must be implemented to act as a source for cached content.
+ * @param <T> type to return
+ */
+public interface ResourceRefresher<T> {
+  /**
+   * Build an up to date version of the data
+   * @return a new instance of the (JSON serializable) data
+   */
+  T refresh();
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
index 8a6b928..f27711a 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
@@ -29,7 +29,6 @@ import org.slf4j.LoggerFactory;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.GET;
-import javax.ws.rs.*;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
@@ -60,16 +59,7 @@ public class ManagementResource extends AbstractSliderResource {
 
   @GET
   public Response getWadl (@Context HttpServletRequest request) {
-    try {
-      URI location = new URL(request.getScheme(),
-                                      request.getServerName(),
-                                      request.getServerPort(),
-                                      "/application.wadl").toURI();
-      return Response.temporaryRedirect(location).build();
-    } catch (Exception e) {
-      log.error("Error during redirect to WADL", e);
-      throw new WebApplicationException(Response.serverError().build());
-    }
+    return redirectToAppWadl(request);
   }
 
   @GET
@@ -100,38 +90,4 @@ public class ManagementResource extends AbstractSliderResource {
   protected AggregateConf getAggregateConf() {
     return slider.getAppState().getInstanceDefinitionSnapshot();
   }
-  
-  @POST
-  @Path("actions/flex") 
-    public void actionFlex() { 
-  }
-  
-  @POST
-  @Path("actions/stop") 
-    public void actionStop() { 
-  }
-  
-    
-  @POST
-  @Path("actions/test/suicide") 
-    public void actionSuicide() { 
-  }
-    
-  @POST
-  @Path("actions/test/kill-container") 
-    public void actionKillContainer() { 
-  }
-
-
-  @GET
-  @Path("containers"+"/components" +"/{name}")
-  public void actionListContainers() {
-  }
-  
-  @GET
-  @Path("containers"+"/components" +"/{name}")
-  public void actionListContainersbyComponent() {
-  }
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
index df19687..431d50f 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
@@ -32,6 +32,7 @@ import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
@@ -39,6 +40,7 @@ import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriInfo;
 import java.io.IOException;
 import java.net.URL;
@@ -81,7 +83,7 @@ public class PublisherResource extends AbstractSliderResource {
     res.setContentType(null);
     log.debug(uriInfo.getRequestUri().toString());
   }
-
+ 
   /**
    * Get a named config set 
    * @param setname name of the config set

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/fad1b978/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
index 73d40ee..4ba0afd 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
@@ -47,8 +47,8 @@ class TestMockContainerResourceAllocations extends BaseMockAppStateTest {
     ConfTree clusterSpec = factory.newConfTree(1, 0, 0)
     ConfTreeOperations cto = new ConfTreeOperations(clusterSpec)
 
-    cto.setRoleOpt(MockRoles.ROLE0, ResourceKeys.YARN_MEMORY, 512)
-    cto.setRoleOpt(MockRoles.ROLE0, ResourceKeys.YARN_CORES, 2)
+    cto.setComponentOpt(MockRoles.ROLE0, ResourceKeys.YARN_MEMORY, 512)
+    cto.setComponentOpt(MockRoles.ROLE0, ResourceKeys.YARN_CORES, 2)
     appState.updateResourceDefinitions(clusterSpec)
     List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
     assert ops.size() == 1
@@ -65,7 +65,7 @@ class TestMockContainerResourceAllocations extends BaseMockAppStateTest {
 
     cto.setComponentOpt(MockRoles.ROLE0, ResourceKeys.YARN_MEMORY,
                            ResourceKeys.YARN_RESOURCE_MAX)
-    cto.setRoleOpt(MockRoles.ROLE0, ResourceKeys.YARN_CORES, 2)
+    cto.setComponentOpt(MockRoles.ROLE0, ResourceKeys.YARN_CORES, 2)
     appState.updateResourceDefinitions(clusterSpec)
     List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
     assert ops.size() == 1
@@ -79,7 +79,7 @@ class TestMockContainerResourceAllocations extends BaseMockAppStateTest {
   public void testMaxCoreAllocations() throws Throwable {
     ConfTree clusterSpec = factory.newConfTree(1, 0, 0)
     ConfTreeOperations cto = new ConfTreeOperations(clusterSpec)
-    cto.setRoleOpt(MockRoles.ROLE0, ResourceKeys.YARN_MEMORY,
+    cto.setComponentOpt(MockRoles.ROLE0, ResourceKeys.YARN_MEMORY,
         512)
     cto.setComponentOpt(MockRoles.ROLE0, ResourceKeys.YARN_CORES,
         ResourceKeys.YARN_RESOURCE_MAX)