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 ji...@apache.org on 2017/06/21 18:33:28 UTC

[06/50] [abbrv] hadoop git commit: YARN-6255. Refactor yarn-native-services framework. Contributed by Jian He

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
index 98557ce..8e8546b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
@@ -21,19 +21,22 @@ package org.apache.slider.server.appmaster.state;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricSet;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.slider.api.types.ComponentInformation;
 import org.apache.slider.api.types.RoleStatistics;
 import org.apache.slider.providers.PlacementPolicy;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.server.appmaster.management.BoolMetricPredicate;
-import org.apache.slider.server.appmaster.management.LongGauge;
+import org.apache.slider.server.appmaster.metrics.SliderMetrics;
 
 import java.io.Serializable;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.apache.hadoop.metrics2.lib.Interns.info;
+
 /**
  * Models the ongoing status of all nodes in an application.
  *
@@ -42,7 +45,7 @@ import java.util.Map;
  * requires synchronization. Where synchronized access is good is that it allows for
  * the whole instance to be locked, for updating multiple entries.
  */
-public final class RoleStatus implements Cloneable, MetricSet {
+public final class RoleStatus implements MetricSet {
 
   private final String name;
   private final String group;
@@ -53,25 +56,9 @@ public final class RoleStatus implements Cloneable, MetricSet {
   private final int key;
   private final ProviderRole providerRole;
 
-  private final LongGauge actual = new LongGauge();
-  private final LongGauge completed = new LongGauge();
-  private final LongGauge desired = new LongGauge();
-  private final LongGauge failed = new LongGauge();
-  private final LongGauge failedRecently = new LongGauge(0);
-  private final LongGauge limitsExceeded = new LongGauge(0);
-  private final LongGauge nodeFailed = new LongGauge(0);
-  /** Number of AA requests queued. */
-  private final LongGauge pendingAntiAffineRequests = new LongGauge(0);
-  private final LongGauge preempted = new LongGauge(0);
-  private final LongGauge releasing = new LongGauge();
-  private final LongGauge requested = new LongGauge();
-  private final LongGauge started = new LongGauge();
-  private final LongGauge startFailed = new LongGauge();
-  private final LongGauge totalRequested = new LongGauge();
-
   /** resource requirements */
   private Resource resourceRequirements;
-
+  private SliderMetrics componentMetrics;
 
   /** any pending AA request */
   private volatile OutstandingRequest outstandingAArequest = null;
@@ -84,28 +71,19 @@ public final class RoleStatus implements Cloneable, MetricSet {
     this.name = providerRole.name;
     this.group = providerRole.group;
     this.key = providerRole.id;
+    componentMetrics =
+        SliderMetrics.register(this.name, "Metrics for component " + this.name);
+    componentMetrics
+        .tag("type", "Metrics type [component or service]", "component");
+  }
+
+  public SliderMetrics getComponentMetrics() {
+    return this.componentMetrics;
   }
 
   @Override
   public Map<String, Metric> getMetrics() {
     Map<String, Metric> metrics = new HashMap<>(15);
-    metrics.put("actual", actual);
-    metrics.put("completed", completed );
-    metrics.put("desired", desired);
-    metrics.put("failed", failed);
-    metrics.put("limitsExceeded", limitsExceeded);
-    metrics.put("nodeFailed", nodeFailed);
-    metrics.put("preempted", preempted);
-    metrics.put("pendingAntiAffineRequests", pendingAntiAffineRequests);
-    metrics.put("releasing", releasing);
-    metrics.put("requested", requested);
-    metrics.put("preempted", preempted);
-    metrics.put("releasing", releasing );
-    metrics.put("requested", requested);
-    metrics.put("started", started);
-    metrics.put("startFailed", startFailed);
-    metrics.put("totalRequested", totalRequested);
-
     metrics.put("outstandingAArequest",
       new BoolMetricPredicate(new BoolMetricPredicate.Eval() {
         @Override
@@ -174,83 +152,6 @@ public final class RoleStatus implements Cloneable, MetricSet {
     return !hasPlacementPolicy(PlacementPolicy.ANYWHERE);
   }
 
-  public long getDesired() {
-    return desired.get();
-  }
-
-  public void setDesired(long desired) {
-    this.desired.set(desired);
-  }
-
-  public long getActual() {
-    return actual.get();
-  }
-
-  public long incActual() {
-    return actual.incrementAndGet();
-  }
-
-  public long decActual() {
-    return actual.decToFloor(1);
-  }
-
-  /**
-   * Get the request count.
-   * @return a count of requested containers
-   */
-  public long getRequested() {
-    return requested.get();
-  }
-
-  public long incRequested() {
-    totalRequested.incrementAndGet();
-    return requested.incrementAndGet();
-  }
-
-  public void cancel(long count) {
-    requested.decToFloor(count);
-  }
-
-  public void decRequested() {
-    cancel(1);
-  }
-
-  public long getReleasing() {
-    return releasing.get();
-  }
-
-  public long incReleasing() {
-    return releasing.incrementAndGet();
-  }
-
-  public long decReleasing() {
-    return releasing.decToFloor(1);
-  }
-
-  public long getFailed() {
-    return failed.get();
-  }
-
-  public long getFailedRecently() {
-    return failedRecently.get();
-  }
-
-  /**
-   * Reset the recent failure
-   * @return the number of failures in the "recent" window
-   */
-  public long resetFailedRecently() {
-    return failedRecently.getAndSet(0);
-  }
-
-  public long getLimitsExceeded() {
-    return limitsExceeded.get();
-  }
-
-  public long incPendingAntiAffineRequests(long v) {
-    return pendingAntiAffineRequests.addAndGet(v);
-  }
-
   /**
    * Probe for an outstanding AA request being true
    * @return true if there is an outstanding AA Request
@@ -271,119 +172,78 @@ public final class RoleStatus implements Cloneable, MetricSet {
    * Note that a role failed, text will
    * be used in any diagnostics if an exception
    * is later raised.
-   * @param startupFailure flag to indicate this was a startup event
    * @param text text about the failure
-   * @param outcome outcome of the container
    */
-  public synchronized void noteFailed(boolean startupFailure, String text,
-      ContainerOutcome outcome) {
+  public synchronized void noteFailed(String text) {
     if (text != null) {
       failureMessage = text;
     }
-    switch (outcome) {
-      case Preempted:
-        preempted.incrementAndGet();
-        break;
-
-      case Node_failure:
-        nodeFailed.incrementAndGet();
-        failed.incrementAndGet();
-        break;
-
-      case Failed_limits_exceeded: // exceeded memory or CPU; app/configuration related
-        limitsExceeded.incrementAndGet();
-        // fall through
-      case Failed: // application failure, possibly node related, possibly not
-      default: // anything else (future-proofing)
-        failed.incrementAndGet();
-        failedRecently.incrementAndGet();
-        //have a look to see if it short lived
-        if (startupFailure) {
-          incStartFailed();
-        }
-        break;
-    }
   }
 
-  public long getStartFailed() {
-    return startFailed.get();
-  }
-
-  public synchronized void incStartFailed() {
-    startFailed.getAndIncrement();
-  }
 
-  public synchronized String getFailureMessage() {
-    return failureMessage;
+  public void setOutstandingAArequest(OutstandingRequest outstandingAArequest) {
+    this.outstandingAArequest = outstandingAArequest;
   }
 
-  public long getCompleted() {
-    return completed.get();
+  /**
+   * Complete the outstanding AA request (there's no check for one in progress, caller
+   * expected to have done that).
+   */
+  public void completeOutstandingAARequest() {
+    setOutstandingAArequest(null);
   }
 
-  public long incCompleted() {
-    return completed.incrementAndGet();
-  }
-  public long getStarted() {
-    return started.get();
+  /**
+   * Cancel any outstanding AA request. Harmless if the role is non-AA, or
+   * if there are no outstanding requests.
+   */
+  public void cancelOutstandingAARequest() {
+    if (outstandingAArequest != null) {
+      setOutstandingAArequest(null);
+    }
   }
 
-  public synchronized void incStarted() {
-    started.incrementAndGet();
+  public long getDesired() {
+    return componentMetrics.containersDesired.value();
   }
 
-  public long getTotalRequested() {
-    return totalRequested.get();
+  long getRunning() {
+    return componentMetrics.containersRunning.value();
   }
 
-  public long getPreempted() {
-    return preempted.get();
+  public long getPending() {
+    return componentMetrics.containersPending.value();
   }
 
-  public long getNodeFailed() {
-    return nodeFailed.get();
+  public long getAAPending() {
+    return componentMetrics.pendingAAContainers.value();
   }
 
-  public long getPendingAntiAffineRequests() {
-    return pendingAntiAffineRequests.get();
+  void decAAPending() {
+    componentMetrics.pendingAAContainers.decr();
   }
-
-  public void setPendingAntiAffineRequests(long pendingAntiAffineRequests) {
-    this.pendingAntiAffineRequests.set(pendingAntiAffineRequests);
+  void setAAPending(long n) {
+    componentMetrics.pendingAAContainers.set((int)n);
   }
 
-  public long decPendingAntiAffineRequests() {
-    return pendingAntiAffineRequests.decToFloor(1);
+  long getFailedRecently() {
+    return componentMetrics.failedSinceLastThreshold.value();
   }
 
-  public OutstandingRequest getOutstandingAArequest() {
-    return outstandingAArequest;
+  long resetFailedRecently() {
+    long count =
+        componentMetrics.failedSinceLastThreshold.value();
+    componentMetrics.failedSinceLastThreshold.set(0);
+    return count;
   }
 
-  public void setOutstandingAArequest(OutstandingRequest outstandingAArequest) {
-    this.outstandingAArequest = outstandingAArequest;
+  long getFailed() {
+    return componentMetrics.containersFailed.value();
   }
 
-  /**
-   * Complete the outstanding AA request (there's no check for one in progress, caller
-   * expected to have done that).
-   */
-  public void completeOutstandingAARequest() {
-    setOutstandingAArequest(null);
-  }
-
-  /**
-   * Cancel any outstanding AA request. Harmless if the role is non-AA, or
-   * if there are no outstanding requests.
-   */
-  public void cancelOutstandingAARequest() {
-    if (outstandingAArequest != null) {
-      setOutstandingAArequest(null);
-      setPendingAntiAffineRequests(0);
-      decRequested();
-    }
+  String getFailureMessage() {
+    return this.failureMessage;
   }
-
   /**
    * Get the number of roles we are short of.
    * nodes released are ignored.
@@ -392,10 +252,9 @@ public final class RoleStatus implements Cloneable, MetricSet {
    */
   public long getDelta() {
     long inuse = getActualAndRequested();
-    long delta = desired.get() - inuse;
+    long delta = getDesired() - inuse;
     if (delta < 0) {
       //if we are releasing, remove the number that are already released.
-      delta += releasing.get();
       //but never switch to a positive
       delta = Math.min(delta, 0);
     }
@@ -407,43 +266,7 @@ public final class RoleStatus implements Cloneable, MetricSet {
    * @return the size of the application when outstanding requests are included.
    */
   public long getActualAndRequested() {
-    return actual.get() + requested.get();
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder("RoleStatus{");
-    sb.append("name='").append(name).append('\'');
-    sb.append(", group=").append(group);
-    sb.append(", key=").append(key);
-    sb.append(", desired=").append(desired);
-    sb.append(", actual=").append(actual);
-    sb.append(", requested=").append(requested);
-    sb.append(", releasing=").append(releasing);
-    sb.append(", failed=").append(failed);
-    sb.append(", startFailed=").append(startFailed);
-    sb.append(", started=").append(started);
-    sb.append(", completed=").append(completed);
-    sb.append(", totalRequested=").append(totalRequested);
-    sb.append(", preempted=").append(preempted);
-    sb.append(", nodeFailed=").append(nodeFailed);
-    sb.append(", failedRecently=").append(failedRecently);
-    sb.append(", limitsExceeded=").append(limitsExceeded);
-    sb.append(", resourceRequirements=").append(resourceRequirements);
-    sb.append(", isAntiAffinePlacement=").append(isAntiAffinePlacement());
-    if (isAntiAffinePlacement()) {
-      sb.append(", pendingAntiAffineRequests=").append(pendingAntiAffineRequests);
-      sb.append(", outstandingAArequest=").append(outstandingAArequest);
-    }
-    sb.append(", failureMessage='").append(failureMessage).append('\'');
-    sb.append(", providerRole=").append(providerRole);
-    sb.append('}');
-    return sb.toString();
-  }
-
-  @Override
-  public synchronized  Object clone() throws CloneNotSupportedException {
-    return super.clone();
+    return getRunning() + getPending();
   }
 
   /**
@@ -455,36 +278,12 @@ public final class RoleStatus implements Cloneable, MetricSet {
   }
 
   /**
-   * Build the statistics map from the current data
-   * @return a map for use in statistics reports
-   */
-  public Map<String, Integer> buildStatistics() {
-    ComponentInformation componentInformation = serialize();
-    return componentInformation.buildStatistics();
-  }
-
-  /**
    * Produced a serialized form which can be served up as JSON
    * @return a summary of the current role status.
    */
   public synchronized ComponentInformation serialize() {
     ComponentInformation info = new ComponentInformation();
     info.name = name;
-    info.priority = getPriority();
-    info.desired = desired.intValue();
-    info.actual = actual.intValue();
-    info.requested = requested.intValue();
-    info.releasing = releasing.intValue();
-    info.failed = failed.intValue();
-    info.startFailed = startFailed.intValue();
-    info.placementPolicy = getPlacementPolicy();
-    info.failureMessage = failureMessage;
-    info.totalRequested = totalRequested.intValue();
-    info.failedRecently = failedRecently.intValue();
-    info.nodeFailed = nodeFailed.intValue();
-    info.preempted = preempted.intValue();
-    info.pendingAntiAffineRequestCount = pendingAntiAffineRequests.intValue();
-    info.isAARequestOutstanding = isAARequestOutstanding();
     return info;
   }
 
@@ -542,17 +341,6 @@ public final class RoleStatus implements Cloneable, MetricSet {
   public synchronized RoleStatistics getStatistics() {
     RoleStatistics stats = new RoleStatistics();
     stats.activeAA = getOutstandingAARequestCount();
-    stats.actual = actual.get();
-    stats.desired = desired.get();
-    stats.failed = failed.get();
-    stats.limitsExceeded = limitsExceeded.get();
-    stats.nodeFailed = nodeFailed.get();
-    stats.preempted = preempted.get();
-    stats.releasing = releasing.get();
-    stats.requested = requested.get();
-    stats.started = started.get();
-    stats.startFailed = startFailed.get();
-    stats.totalRequested = totalRequested.get();
     return stats;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
index ad91183..118ca9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
@@ -21,15 +21,13 @@ package org.apache.slider.server.appmaster.state;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.api.ClusterNode;
 import org.apache.slider.api.StatusKeys;
+import org.apache.slider.api.resource.Application;
 import org.apache.slider.api.types.ApplicationLivenessInformation;
 import org.apache.slider.api.types.ComponentInformation;
 import org.apache.slider.api.types.NodeInformation;
 import org.apache.slider.api.types.RoleStatistics;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.conf.ConfTreeOperations;
 import org.apache.slider.core.exceptions.NoSuchNodeException;
 import org.apache.slider.core.registry.docstore.PublishedConfigSet;
 import org.apache.slider.core.registry.docstore.PublishedExportsSet;
@@ -105,29 +103,7 @@ public interface StateAccessForProviders {
    * Get the current cluster description 
    * @return the actual state of the cluster
    */
-  ClusterDescription getClusterStatus();
-
-  /**
-   * Get at the snapshot of the resource config
-   * Changes here do not affect the application state.
-   * @return the most recent settings
-   */
-  ConfTreeOperations getResourcesSnapshot();
-
-  /**
-   * Get at the snapshot of the appconf config
-   * Changes here do not affect the application state.
-   * @return the most recent settings
-   */
-  ConfTreeOperations getAppConfSnapshot();
-
-  /**
-   * Get at the snapshot of the internals config.
-   * Changes here do not affect the application state.
-   * @return the internals settings
-   */
-
-  ConfTreeOperations getInternalsSnapshot();
+  Application getApplication();
 
   /**
    * Flag set to indicate the application is live -this only happens
@@ -135,22 +111,8 @@ public interface StateAccessForProviders {
    */
   boolean isApplicationLive();
 
-  long getSnapshotTime();
-
-  /**
-   * Get a snapshot of the entire aggregate configuration
-   * @return the aggregate configuration
-   */
-  AggregateConf getInstanceDefinitionSnapshot();
-
-  /**
-   * Get the desired/unresolved value
-   * @return unresolved
-   */
-  AggregateConf getUnresolvedInstanceDefinition();
-
   /**
-   * Look up a role from its key -or fail 
+   * Look up a role from its key -or fail
    *
    * @param key key to resolve
    * @return the status
@@ -159,7 +121,7 @@ public interface StateAccessForProviders {
   RoleStatus lookupRoleStatus(int key);
 
   /**
-   * Look up a role from its key -or fail 
+   * Look up a role from its key -or fail
    *
    * @param c container in a role
    * @return the status
@@ -232,14 +194,8 @@ public interface StateAccessForProviders {
   /**
    * Update the cluster description with anything interesting
    */
-  ClusterDescription refreshClusterStatus();
+  Application refreshClusterStatus();
 
-  /**
-   * Get a deep clone of the role status list. Concurrent events may mean this
-   * list (or indeed, some of the role status entries) may be inconsistent
-   * @return a snapshot of the role status entries
-   */
-  List<RoleStatus> cloneRoleStatusList();
 
   /**
    * get application liveness information
@@ -248,13 +204,6 @@ public interface StateAccessForProviders {
   ApplicationLivenessInformation getApplicationLivenessInformation();
 
   /**
-   * Get the live statistics map
-   * @return a map of statistics values, defined in the {@link StatusKeys}
-   * keylist.
-   */
-  Map<String, Integer> getLiveStatistics();
-
-  /**
    * Get a snapshot of component information.
    * <p>
    *   This does <i>not</i> include any container list, which 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
index 7ecc00c..0cac430 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
@@ -16,6 +16,8 @@
  */
 package org.apache.slider.server.appmaster.web;
 
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.health.HealthCheckRegistry;
 import com.codahale.metrics.servlets.HealthCheckServlet;
 import com.codahale.metrics.servlets.MetricsServlet;
 import com.codahale.metrics.servlets.PingServlet;
@@ -28,10 +30,8 @@ import com.sun.jersey.spi.container.servlet.ServletContainer;
 import org.apache.hadoop.yarn.webapp.Dispatcher;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebApp;
-import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 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;
@@ -39,6 +39,8 @@ import java.util.Map;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
 /**
  * 
  */
@@ -65,6 +67,7 @@ public class SliderAMWebApp extends WebApp {
     bind(GenericExceptionHandler.class);
     // bind the REST interface
     bind(AMWebServices.class);
+
     //bind(AMAgentWebServices.class);
     route("/", SliderAMController.class);
     route(CONTAINER_STATS, SliderAMController.class, "containerStats");
@@ -81,11 +84,9 @@ public class SliderAMWebApp extends WebApp {
       serve(path).with(Dispatcher.class);
     }
 
-    // metrics
-    MetricsAndMonitoring monitoring =
-        webAppApi.getMetricsAndMonitoring();
-    serve(SYSTEM_HEALTHCHECK).with(new HealthCheckServlet(monitoring.getHealth()));
-    serve(SYSTEM_METRICS).with(new MetricsServlet(monitoring.getMetrics()));
+    serve(SYSTEM_HEALTHCHECK)
+        .with(new HealthCheckServlet(new HealthCheckRegistry()));
+    serve(SYSTEM_METRICS).with(new MetricsServlet(new MetricRegistry()));
     serve(SYSTEM_PING).with(new PingServlet());
     serve(SYSTEM_THREADS).with(new ThreadDumpServlet());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
index ea07a8a..094726d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
@@ -22,7 +22,6 @@ import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 import org.apache.slider.server.appmaster.state.AppState;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
 
 /**
  * Interface to pass information from the Slider AppMaster to the WebApp
@@ -56,10 +55,4 @@ public interface WebAppApi {
    * @return the immediate and scheduled queues
    */
   QueueAccess getQueues();
-
-  /**
-   * Local cache of content
-   * @return the cache
-   */
-  ContentCache getContentCache();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
index d20f1ad..fd9381c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
@@ -18,11 +18,9 @@ package org.apache.slider.server.appmaster.web;
 
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.slider.providers.ProviderService;
-import org.apache.slider.server.appmaster.AppMasterActionOperations;
 import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,15 +37,10 @@ public class WebAppApiImpl implements WebAppApi {
   private final RegistryOperations registryOperations;
   private final MetricsAndMonitoring metricsAndMonitoring;
   private final QueueAccess queues;
-  private final AppMasterActionOperations appMasterOperations;
-  private final ContentCache contentCache;
 
   public WebAppApiImpl(StateAccessForProviders appState,
       ProviderService provider, RegistryOperations registryOperations,
-      MetricsAndMonitoring metricsAndMonitoring, QueueAccess queues,
-      AppMasterActionOperations appMasterOperations, ContentCache contentCache) {
-    this.appMasterOperations = appMasterOperations;
-    this.contentCache = contentCache;
+      MetricsAndMonitoring metricsAndMonitoring, QueueAccess queues) {
     checkNotNull(appState);
     checkNotNull(provider);
     this.queues = queues;
@@ -82,10 +75,4 @@ public class WebAppApiImpl implements WebAppApi {
   public QueueAccess getQueues() {
     return queues;
   }
-
-
-  @Override
-  public ContentCache getContentCache() {
-    return contentCache;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
index aed87d8..e73dd87 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
@@ -18,13 +18,24 @@ package org.apache.slider.server.appmaster.web.rest;
 
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
+import org.apache.slider.api.resource.Application;
 import org.apache.slider.server.appmaster.web.WebAppApi;
-import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource;
+import org.apache.slider.server.appmaster.web.rest.application.actions.RestActionStop;
+import org.apache.slider.server.appmaster.web.rest.application.actions.StopResponse;
 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.servlet.http.HttpServletRequest;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
 import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.UriInfo;
+
+import static javax.ws.rs.core.MediaType.APPLICATION_JSON;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.ACTION_STOP;
 
 /**
  *  The available REST services exposed by a slider AM. 
@@ -38,7 +49,6 @@ public class AMWebServices {
   private final ManagementResource managementResource;
   private final PublisherResource publisherResource;
   private final RegistryResource registryResource;
-  private final ApplicationResource applicationResource;
 
   @Inject
   public AMWebServices(WebAppApi slider) {
@@ -46,7 +56,6 @@ public class AMWebServices {
     managementResource = new ManagementResource(slider);
     publisherResource = new PublisherResource(slider);
     registryResource = new RegistryResource(slider);
-    applicationResource = new ApplicationResource(slider);
   }
 
   @Path(RestPaths.SLIDER_SUBPATH_MANAGEMENT)
@@ -63,9 +72,21 @@ public class AMWebServices {
   public RegistryResource getRegistryResource() {
     return registryResource;
   }
-  
+
+
+  @GET
   @Path(RestPaths.SLIDER_SUBPATH_APPLICATION)
-  public ApplicationResource getApplicationResource() {
-    return applicationResource;
+  @Produces({APPLICATION_JSON})
+  public Application getApplicationResource() {
+    return slider.getAppState().getApplication();
+  }
+
+  @POST
+  @Path(ACTION_STOP)
+  @Produces({APPLICATION_JSON})
+  public StopResponse actionStop(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo,
+      String body) {
+    return new RestActionStop(slider).stop(request, uriInfo, body);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
index ae9eb0f..581f5b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
@@ -34,11 +34,8 @@ public class RestPaths {
   /**
    * agent content root: {@value}
    */
-  public static final String WS_AGENT_CONTEXT_ROOT = "/" + AGENT_WS_CONTEXT;
-  public static final String V1_SLIDER = "/v1/slider";
-  public static final String SLIDER_CONTEXT_ROOT = WS_CONTEXT_ROOT + V1_SLIDER;
-  public static final String RELATIVE_API = WS_CONTEXT + V1_SLIDER;
-  public static final String SLIDER_AGENT_CONTEXT_ROOT = WS_AGENT_CONTEXT_ROOT + V1_SLIDER;
+  public static final String SLIDER_CONTEXT_ROOT = WS_CONTEXT_ROOT + "/v1";
+  public static final String RELATIVE_API = WS_CONTEXT + "/v1";
   public static final String MANAGEMENT = "mgmt";
   public static final String SLIDER_SUBPATH_MANAGEMENT = "/" + MANAGEMENT;
   public static final String SLIDER_SUBPATH_AGENTS = "/agents";
@@ -46,21 +43,6 @@ public class RestPaths {
 
 
   /**
-   * management path: {@value}
-   */
-  public static final String SLIDER_PATH_MANAGEMENT = SLIDER_CONTEXT_ROOT
-                                      + SLIDER_SUBPATH_MANAGEMENT;
-
-  public static final String RELATIVE_PATH_MANAGEMENT = RELATIVE_API
-                                      + SLIDER_SUBPATH_MANAGEMENT;
-
-  /**
-   * Agents: {@value}
-   */
-  public static final String SLIDER_PATH_AGENTS = SLIDER_AGENT_CONTEXT_ROOT
-                                      + SLIDER_SUBPATH_AGENTS;
-  
-  /**
    * Publisher: {@value}
    */
   public static final String SLIDER_PATH_PUBLISHER = SLIDER_CONTEXT_ROOT
@@ -105,6 +87,7 @@ public class RestPaths {
   public static final String SYSTEM = "/system";
 
 
+
   /**
    * Codahale Metrics - health: {@value}
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java
index 2facf16..d23fcee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java
@@ -19,17 +19,7 @@
 package org.apache.slider.server.appmaster.web.rest.application;
 
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.rest.application.resources.AggregateModelRefresher;
-import org.apache.slider.server.appmaster.web.rest.application.resources.AppconfRefresher;
-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.LiveComponentsRefresher;
-import org.apache.slider.server.appmaster.web.rest.application.resources.LiveContainersRefresher;
-import org.apache.slider.server.appmaster.web.rest.application.resources.LiveNodesRefresher;
-import org.apache.slider.server.appmaster.web.rest.application.resources.LiveResourcesRefresher;
-import org.apache.slider.server.appmaster.web.rest.application.resources.LiveStatisticsRefresher;
-
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
 
 public class ApplicationResouceContentCacheFactory {
   public static final int LIFESPAN = 500;
@@ -41,23 +31,6 @@ public class ApplicationResouceContentCacheFactory {
   public static ContentCache createContentCache(
       StateAccessForProviders state) {
     ContentCache cache = new ContentCache();
-    cache.put(LIVE_RESOURCES, new CachedContent<>(LIFESPAN, new LiveResourcesRefresher(state)));
-    cache.put(LIVE_CONTAINERS, new CachedContent<>(LIFESPAN, new LiveContainersRefresher(state)));
-    cache.put(LIVE_COMPONENTS, new CachedContent<>(LIFESPAN, new LiveComponentsRefresher(state)));
-    cache.put(LIVE_NODES, new CachedContent<>(LIFESPAN, new LiveNodesRefresher(state)));
-    cache.put(MODEL_DESIRED,
-        new CachedContent<>(LIFESPAN, new AggregateModelRefresher(state, false)));
-    cache.put(MODEL_RESOLVED,
-        new CachedContent<>(LIFESPAN, new AggregateModelRefresher(state, true)));
-    cache.put(MODEL_RESOLVED_APPCONF,
-        new CachedContent<>(LIFESPAN, new AppconfRefresher(state, false, false)));
-    cache.put(MODEL_RESOLVED_RESOURCES,
-        new CachedContent<>(LIFESPAN, new AppconfRefresher(state, false, true)));
-    cache.put(MODEL_DESIRED_APPCONF,
-        new CachedContent<>(LIFESPAN, new AppconfRefresher(state, true, false)));
-    cache.put(MODEL_DESIRED_RESOURCES,
-        new CachedContent<>(LIFESPAN, new AppconfRefresher(state, true, true)));
-    cache.put(LIVE_STATISTICS, new CachedContent<>(LIFESPAN, new LiveStatisticsRefresher(state)));
     return cache;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
deleted file mode 100644
index 52068d6..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
+++ /dev/null
@@ -1,516 +0,0 @@
-/*
- * 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.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.webapp.BadRequestException;
-import org.apache.hadoop.yarn.webapp.NotFoundException;
-import org.apache.slider.api.types.ApplicationLivenessInformation;
-import org.apache.slider.api.types.ComponentInformation;
-import org.apache.slider.api.types.ContainerInformation;
-import org.apache.slider.api.types.NodeInformation;
-import org.apache.slider.api.types.NodeInformationList;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.conf.ConfTree;
-import org.apache.slider.core.exceptions.NoSuchNodeException;
-import org.apache.slider.core.persist.ConfTreeSerDeser;
-import org.apache.slider.server.appmaster.actions.ActionFlexCluster;
-import org.apache.slider.server.appmaster.actions.AsyncAction;
-import org.apache.slider.server.appmaster.actions.QueueAccess;
-import org.apache.slider.server.appmaster.state.RoleInstance;
-import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.WebAppApi;
-import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
-
-import org.apache.slider.server.appmaster.web.rest.application.actions.RestActionStop;
-import org.apache.slider.server.appmaster.web.rest.application.actions.StopResponse;
-import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
-import org.apache.slider.server.appmaster.web.rest.application.actions.RestActionPing;
-import org.apache.slider.api.types.PingInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.inject.Singleton;
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.HEAD;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Context;
-
-import static javax.ws.rs.core.MediaType.*;
-
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.UriInfo;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-@Singleton
-@SuppressWarnings("unchecked")
-public class ApplicationResource extends AbstractSliderResource {
-  private static final Logger log =
-      LoggerFactory.getLogger(ApplicationResource.class);
-
-  public static final List<String> LIVE_ENTRIES = toJsonList("resources",
-      "containers",
-      "components",
-      "nodes",
-      "statistics",
-      "internal");
-
-  public static final List<String> ROOT_ENTRIES =
-      toJsonList("model", "live", "actions");
-
-  public static final List<String> MODEL_ENTRIES =
-      toJsonList("desired", "resolved");
-
-  /**
-   * This is the cache of all content ... each entry is
-   * designed to be self-refreshing on get operations, 
-   * so is never very out of date, yet many GETs don't
-   * overload the rest of the system.
-   */
-  private final ContentCache cache;
-  private final StateAccessForProviders state;
-  private final QueueAccess actionQueues;
-
-  public ApplicationResource(WebAppApi slider) {
-    super(slider);
-    state = slider.getAppState();
-    cache = slider.getContentCache();
-    actionQueues = slider.getQueues();
-  }
-
-  /**
-   * Build a new JSON-marshallable list of string elements
-   * @param elements elements
-   * @return something that can be returned
-   */
-  private static List<String> toJsonList(String... elements) {
-    return Lists.newArrayList(elements);
-  }
-
-  @GET
-  @Path("/")
-  @Produces({APPLICATION_JSON})
-  public List<String> getRoot() {
-    markGet(SLIDER_SUBPATH_APPLICATION);
-    return ROOT_ENTRIES;
-  }
-
-  /**
-   * Enum model values: desired and resolved
-   * @return the desired and resolved model
-   */
-  @GET
-  @Path(MODEL)
-  @Produces({APPLICATION_JSON})
-  public List<String> getModel() {
-    markGet(SLIDER_SUBPATH_APPLICATION, MODEL);
-    return MODEL_ENTRIES;
-  }
-
-  @GET
-  @Path(MODEL_DESIRED)
-  @Produces({APPLICATION_JSON})
-  public AggregateConf getModelDesired() {
-    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED);
-    return lookupAggregateConf(MODEL_DESIRED);
-  }
-  
-  @GET
-  @Path(MODEL_DESIRED_APPCONF)
-  @Produces({APPLICATION_JSON})
-  public ConfTree getModelDesiredAppconf() {
-    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED_APPCONF);
-    return lookupConfTree(MODEL_DESIRED_APPCONF);
-  }
-
-  @GET
-  @Path(MODEL_DESIRED_RESOURCES)
-  @Produces({APPLICATION_JSON})
-  public ConfTree getModelDesiredResources() {
-    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED_RESOURCES);
-    return lookupConfTree(MODEL_DESIRED_RESOURCES);
-  }
-
-/*
-  @PUT
-  @Path(MODEL_DESIRED_RESOURCES)
-//  @Consumes({APPLICATION_JSON, TEXT_PLAIN})
-  @Consumes({TEXT_PLAIN})
-  @Produces({APPLICATION_JSON})
-*/
-  public ConfTree setModelDesiredResources(
-      String json) {
-    markPut(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED_RESOURCES);
-    int size = json != null ? json.length() : 0;
-    log.info("PUT {} {} bytes:\n{}", MODEL_DESIRED_RESOURCES,
-        size,
-        json);
-    if (size == 0) {
-      log.warn("No JSON in PUT request; rejecting");
-      throw new BadRequestException("No JSON in PUT");
-    }
-    
-    try {
-      ConfTreeSerDeser serDeser = new ConfTreeSerDeser();
-      ConfTree updated = serDeser.fromJson(json);
-      queue(new ActionFlexCluster("flex",
-          1, TimeUnit.MILLISECONDS,
-          updated));
-      // return the updated value, even though it potentially hasn't yet
-      // been executed
-      return updated;
-    } catch (Exception e) {
-      throw buildException("PUT to "+ MODEL_DESIRED_RESOURCES , e);
-    }
-  }
-  @PUT
-  @Path(MODEL_DESIRED_RESOURCES)
-  @Consumes({APPLICATION_JSON})
-  @Produces({APPLICATION_JSON})
-  public ConfTree setModelDesiredResources(
-      ConfTree updated) {
-    try {
-      queue(new ActionFlexCluster("flex",
-          1, TimeUnit.MILLISECONDS,
-          updated));
-      // return the updated value, even though it potentially hasn't yet
-      // been executed
-      return updated;
-    } catch (Exception e) {
-      throw buildException("PUT to "+ MODEL_DESIRED_RESOURCES , e);
-    }
-  }
-  
-  
-
-  @GET
-  @Path(MODEL_RESOLVED)
-  @Produces({APPLICATION_JSON})
-  public AggregateConf getModelResolved() {
-    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED);
-    return lookupAggregateConf(MODEL_RESOLVED);
-  }
-
-  @GET
-  @Path(MODEL_RESOLVED_APPCONF)
-  @Produces({APPLICATION_JSON})
-  public ConfTree getModelResolvedAppconf() {
-    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED_APPCONF);
-    return lookupConfTree(MODEL_RESOLVED_APPCONF);
-  }
-
-  @GET
-  @Path(MODEL_RESOLVED_RESOURCES)
-  @Produces({APPLICATION_JSON})
-  public ConfTree getModelResolvedResources() {
-    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED_RESOURCES);
-    return lookupConfTree(MODEL_RESOLVED_RESOURCES);
-  }
-  
-  @GET
-  @Path(LIVE)
-  @Produces({APPLICATION_JSON})
-  public List<String> getLive() {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE);
-    return LIVE_ENTRIES;
-  }
-
-  @GET
-  @Path(LIVE_RESOURCES)
-  @Produces({APPLICATION_JSON})
-  public ConfTree getLiveResources() {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_RESOURCES);
-    return lookupConfTree(LIVE_RESOURCES);
-  }
-  
-  @GET
-  @Path(LIVE_CONTAINERS)
-  @Produces({APPLICATION_JSON})
-  public Map<String, ContainerInformation> getLiveContainers() {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_CONTAINERS);
-    try {
-      return (Map<String, ContainerInformation>)cache.lookup(
-          LIVE_CONTAINERS);
-    } catch (Exception e) {
-      throw buildException(LIVE_CONTAINERS, e);
-    }
-  }
-
-  @GET
-  @Path(LIVE_CONTAINERS + "/{containerId}")
-  @Produces({APPLICATION_JSON})
-  public ContainerInformation getLiveContainer(
-      @PathParam("containerId") String containerId) {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_CONTAINERS);
-    try {
-      RoleInstance id = state.getLiveInstanceByContainerID(containerId);
-      return id.serialize();
-    } catch (NoSuchNodeException e) {
-      throw new NotFoundException("Unknown container: " + containerId);
-    } catch (Exception e) {
-      throw buildException(LIVE_CONTAINERS + "/"+ containerId, e);
-    }
-  }
-
-  @GET
-  @Path(LIVE_COMPONENTS)
-  @Produces({APPLICATION_JSON})
-  public Map<String, ComponentInformation> getLiveComponents() {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
-    try {
-      return (Map<String, ComponentInformation>) cache.lookup(LIVE_COMPONENTS);
-    } catch (Exception e) {
-      throw buildException(LIVE_COMPONENTS, e);
-    }
-  }
-  
-  @GET
-  @Path(LIVE_COMPONENTS + "/{component}")
-  @Produces({APPLICATION_JSON})
-  public ComponentInformation getLiveComponent(
-      @PathParam("component") String component) {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
-    try {
-      return state.getComponentInformation(component);
-    } catch (YarnRuntimeException e) {
-      throw new NotFoundException("Unknown component: " + component);
-    } catch (Exception e) {
-      throw buildException(LIVE_CONTAINERS +"/" + component, e);
-    }
-  }
-
-  /**
-   * Liveness information for the application as a whole
-   * @return snapshot of liveness
-   */
-  @GET
-  @Path(LIVE_LIVENESS)
-  @Produces({APPLICATION_JSON})
-  public ApplicationLivenessInformation getLivenessInformation() {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_LIVENESS);
-    try {
-      return state.getApplicationLivenessInformation();
-    } catch (Exception e) {
-      throw buildException(LIVE_CONTAINERS, e);
-    }
-  }
-
-/*
-TODO: decide what structure to return here, then implement
-
-  @GET
-  @Path(LIVE_LIVENESS + "/{component}")
-  @Produces({APPLICATION_JSON})
-  public ApplicationLivenessInformation getLivenessForComponent(
-      @PathParam("component") String component) {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
-    try {
-      RoleStatus roleStatus = state.lookupRoleStatus(component);
-      ApplicationLivenessInformation info = new ApplicationLivenessInformation();
-      info.requested = roleStatus.getRequested();
-      info.allRequestsSatisfied = info.requested == 0;
-      return info;
-    } catch (YarnRuntimeException e) {
-      throw new NotFoundException("Unknown component: " + component);
-    } catch (Exception e) {
-      throw buildException(LIVE_LIVENESS + "/" + component, e);
-    }
-  }
-*/
-
-
-  @GET
-  @Path(LIVE_NODES)
-  @Produces({APPLICATION_JSON})
-  public NodeInformationList getLiveNodes() {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
-    try {
-      return (NodeInformationList) cache.lookup(LIVE_NODES);
-    } catch (Exception e) {
-      throw buildException(LIVE_COMPONENTS, e);
-    }
-  }
-
-  @GET
-  @Path(LIVE_NODES + "/{hostname}")
-  @Produces({APPLICATION_JSON})
-  public NodeInformation getLiveNode(@PathParam("hostname") String hostname) {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
-    try {
-      NodeInformation ni = state.getNodeInformation(hostname);
-      if (ni != null) {
-        return ni;
-      } else {
-        throw new NotFoundException("Unknown node: " + hostname);
-      }
-    } catch (NotFoundException e) {
-      throw e;
-    } catch (Exception e) {
-      throw buildException(LIVE_COMPONENTS + "/" + hostname, e);
-    }
-  }
-
-  /**
-   * Statistics of the application
-   * @return snapshot statistics
-   */
-  @GET
-  @Path(LIVE_STATISTICS)
-  @Produces({APPLICATION_JSON})
-  public Map<String, Integer> getLiveStatistics() {
-    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_LIVENESS);
-    try {
-      return (Map<String, Integer>) cache.lookup(LIVE_STATISTICS);
-    } catch (Exception e) {
-      throw buildException(LIVE_STATISTICS, e);
-    }
-  }
-
-  /**
-   * Helper method; look up an aggregate configuration in the cache from
-   * a key, or raise an exception
-   * @param key key to resolve
-   * @return the configuration
-   * @throws WebApplicationException on a failure
-   */
-  protected AggregateConf lookupAggregateConf(String key) {
-    try {
-      return (AggregateConf) cache.lookup(key);
-    } catch (Exception e) {
-      throw buildException(key, e);
-    }
-  }
-
-
-  /**
-   * Helper method; look up an conf tree in the cache from
-   * a key, or raise an exception
-   * @param key key to resolve
-   * @return the configuration
-   * @throws WebApplicationException on a failure
-   */
-  protected ConfTree lookupConfTree(String key) {
-    try {
-      return (ConfTree) cache.lookup(key);
-    } catch (Exception e) {
-      throw buildException(key, e);
-    }
-  }
-
-  /* ************************************************************************
-  
-  ACTION PING
-  
-  **************************************************************************/
-  
-  @GET
-  @Path(ACTION_PING)
-  @Produces({APPLICATION_JSON})
-  public PingInformation actionPingGet(@Context HttpServletRequest request,
-      @Context UriInfo uriInfo) {
-    markGet(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
-    return new RestActionPing().ping(request, uriInfo, "");
-  }
-  
-  @POST
-  @Path(ACTION_PING)
-  @Produces({APPLICATION_JSON})
-  public PingInformation actionPingPost(@Context HttpServletRequest request,
-      @Context UriInfo uriInfo,
-      String body) {
-    markPost(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
-    return new RestActionPing().ping(request, uriInfo, body);
-  }
-  
-  @PUT
-  @Path(ACTION_PING)
-  @Consumes({TEXT_PLAIN})
-  @Produces({APPLICATION_JSON})
-  public PingInformation actionPingPut(@Context HttpServletRequest request,
-      @Context UriInfo uriInfo,
-      String body) {
-    markPut(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
-    return new RestActionPing().ping(request, uriInfo, body);
-  }
-  
-  @DELETE
-  @Path(ACTION_PING)
-  @Consumes({APPLICATION_JSON})
-  @Produces({APPLICATION_JSON})
-  public PingInformation actionPingDelete(@Context HttpServletRequest request,
-      @Context UriInfo uriInfo) {
-    markDelete(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
-    return new RestActionPing().ping(request, uriInfo, "");
-  }
-  
-  @HEAD
-  @Path(ACTION_PING)
-  public Object actionPingHead(@Context HttpServletRequest request,
-      @Context UriInfo uriInfo) {
-    mark("HEAD", SLIDER_SUBPATH_APPLICATION, ACTION_PING);
-    return new RestActionPing().ping(request, uriInfo, "");
-  }
-  
-  /* ************************************************************************
-  
-  ACTION STOP
-  
-  **************************************************************************/
-
-
-  @POST
-  @Path(ACTION_STOP)
-  @Produces({APPLICATION_JSON})
-  public StopResponse actionStop(@Context HttpServletRequest request,
-      @Context UriInfo uriInfo,
-      String body) {
-    markPost(SLIDER_SUBPATH_APPLICATION, ACTION_STOP);
-    return new RestActionStop(slider).stop(request, uriInfo, body);
-  }
-
-  /**
-   * Schedule an action
-   * @param action for delayed execution
-   */
-  public void schedule(AsyncAction action) {
-    actionQueues.schedule(action);
-  }
-
-  /**
-   * Put an action on the immediate queue -to be executed when the queue
-   * reaches it.
-   * @param action action to queue
-   */
-  public void queue(AsyncAction action) {
-    actionQueues.put(action);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
index ee28abf..261e66e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
@@ -23,7 +23,6 @@ import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 
 /**
  * Refresh the aggregate desired model via
- * {@link StateAccessForProviders#getInstanceDefinitionSnapshot()}
  */
 public class AggregateModelRefresher
     implements ResourceRefresher<AggregateConf> {
@@ -39,9 +38,6 @@ public class AggregateModelRefresher
 
   @Override
   public AggregateConf refresh() throws Exception {
-    return
-        resolved ?
-          state.getInstanceDefinitionSnapshot()
-          : state.getUnresolvedInstanceDefinition();
+    return new AggregateConf();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
index 06460cc..190a51e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
@@ -44,10 +44,7 @@ public class AppconfRefresher
 
   @Override
   public ConfTree refresh() throws Exception {
-    AggregateConf aggregateConf =
-        unresolved ?
-        state.getUnresolvedInstanceDefinition():
-        state.getInstanceDefinitionSnapshot();
+    AggregateConf aggregateConf = new AggregateConf();
     ConfTree ct = resources ? aggregateConf.getResources() 
                             : aggregateConf.getAppConf();
     return new ConfTreeSerDeser().fromInstance(ct);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
deleted file mode 100644
index f988297..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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() throws Exception {
-
-    // snapshot resources
-    ConfTreeOperations resources = state.getResourcesSnapshot();
-    // then add actual values
-    Map<Integer, RoleStatus> roleStatusMap = state.getRoleStatusMap();
-    
-    for (RoleStatus status : roleStatusMap.values()) {
-      String name = status.getName();
-      resources.setComponentOpt(name,
-          StatusKeys.COMPONENT_INSTANCES_REQUESTING,
-          status.getRequested());
-      resources.setComponentOpt(name,
-          StatusKeys.COMPONENT_INSTANCES_ACTUAL,
-          status.getActual());
-      resources.setComponentOpt(name,
-          StatusKeys.COMPONENT_INSTANCES_RELEASING,
-          status.getReleasing());
-      resources.setComponentOpt(name,
-          StatusKeys.COMPONENT_INSTANCES_FAILED,
-          status.getFailed());
-      resources.setComponentOpt(name,
-          StatusKeys.COMPONENT_INSTANCES_COMPLETED,
-          status.getCompleted());
-      resources.setComponentOpt(name,
-          StatusKeys.COMPONENT_INSTANCES_STARTED,
-          status.getStarted());
-    }
-    return resources.getConfTree();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveStatisticsRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveStatisticsRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveStatisticsRefresher.java
deleted file mode 100644
index d31b455..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveStatisticsRefresher.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.server.appmaster.state.StateAccessForProviders;
-
-import java.util.Map;
-
-public class LiveStatisticsRefresher implements ResourceRefresher<Map<String,Integer>> {
-
-  private final StateAccessForProviders state;
-
-  public LiveStatisticsRefresher(StateAccessForProviders state) {
-    this.state = state;
-  }
-
-  @Override
-  public Map<String, Integer> refresh() throws Exception {
-
-    // snapshot resources
-    return state.getLiveStatistics();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
deleted file mode 100644
index c16912a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.core.conf.ConfTree;
-import org.apache.slider.core.conf.ConfTreeOperations;
-import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-
-public class ResourceSnapshotRefresher implements ResourceRefresher<ConfTree> {
-
-  private final StateAccessForProviders state;
-
-  public ResourceSnapshotRefresher(StateAccessForProviders state) {
-    this.state = state;
-  }
-
-  @Override
-  public ConfTree refresh() throws Exception {
-
-    // snapshot resources
-    ConfTreeOperations resources = state.getResourcesSnapshot();
-      return resources.getConfTree();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
index f27711a..14d9400 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
@@ -88,6 +88,7 @@ public class ManagementResource extends AbstractSliderResource {
   }
 
   protected AggregateConf getAggregateConf() {
-    return slider.getAppState().getInstanceDefinitionSnapshot();
+    //TODO
+    return  new AggregateConf();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java
index 2f02f27..79b687f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java
@@ -49,7 +49,7 @@ public class ClusterSpecificationBlock extends SliderHamletBlock {
    * @return
    */
   private String getJson() {
-    return appState.getClusterStatus().toString();
+    return appState.getApplication().toString();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
index 8b7d695..4796d6c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
-import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.resource.Application;
 import org.apache.slider.api.types.ComponentInformation;
 import org.apache.slider.server.appmaster.state.RoleInstance;
 import org.apache.slider.server.appmaster.web.WebAppApi;
@@ -141,18 +141,10 @@ public class ContainerStatsBlock extends SliderHamletBlock {
 
           }));
 
-      ClusterDescription desc = appState.getClusterStatus();
-      Map<String, String> options = desc.getRole(name);
+      Application application = appState.getApplication();
       Iterable<Entry<TableContent, String>> tableContent;
-      
-      // Generate the pairs of data in the expected form
-      if (null != options) {
-        tableContent = Iterables.transform(options.entrySet(), stringStringPairFunc);
-      } else {
-        // Or catch that we have no options and provide "empty"
-        tableContent = Collections.emptySet();
-      }
-      
+      tableContent = Collections.emptySet();
+
       // Generate the options used by this role
       generateRoleDetails(div, "role-options-wrap", "Role Options", tableContent);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
index 2f99b27..440094e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
@@ -22,15 +22,12 @@ import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.LI;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL;
-import org.apache.slider.api.ClusterDescription;
-import org.apache.slider.api.StatusKeys;
 import org.apache.slider.api.types.ApplicationLivenessInformation;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.registry.docstore.ExportEntry;
 import org.apache.slider.core.registry.docstore.PublishedExports;
 import org.apache.slider.core.registry.docstore.PublishedExportsSet;
-import org.apache.slider.providers.MonitorDetail;
-import org.apache.slider.providers.ProviderService;
+import org.apache.slider.server.appmaster.metrics.SliderMetrics;
 import org.apache.slider.server.appmaster.state.RoleStatus;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.slf4j.Logger;
@@ -39,7 +36,6 @@ import org.slf4j.LoggerFactory;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
@@ -71,8 +67,7 @@ public class IndexBlock extends SliderHamletBlock {
   // An extra method to make testing easier since you can't make an instance of Block
   @VisibleForTesting
   protected void doIndex(Hamlet html, String providerName) {
-    ClusterDescription clusterStatus = appState.getClusterStatus();
-    String name = clusterStatus.name;
+    String name = appState.getApplicationName();
     if (name != null && (name.startsWith(" ") || name.endsWith(" "))) {
       name = "'" + name + "'";
     } 
@@ -96,23 +91,23 @@ public class IndexBlock extends SliderHamletBlock {
           ._();
     table1.tr()
           .td("Create time: ")
-          .td(getInfoAvoidingNulls(StatusKeys.INFO_CREATE_TIME_HUMAN))
+          .td("N/A")
           ._();
     table1.tr()
           .td("Running since: ")
-          .td(getInfoAvoidingNulls(StatusKeys.INFO_LIVE_TIME_HUMAN))
+          .td("N/A")
           ._();
     table1.tr()
           .td("Time last flexed: ")
-          .td(getInfoAvoidingNulls(StatusKeys.INFO_FLEX_TIME_HUMAN))
+          .td("N/A")
           ._();
     table1.tr()
           .td("Application storage path: ")
-          .td(clusterStatus.dataPath)
+          .td("N/A")
           ._();
     table1.tr()
           .td("Application configuration path: ")
-          .td(clusterStatus.originConfigurationPath)
+          .td("N/A")
           ._();
     table1._();
     div._();
@@ -136,7 +131,8 @@ public class IndexBlock extends SliderHamletBlock {
     trb(header, "Placement");
     header._()._();  // tr & thead
 
-    List<RoleStatus> roleStatuses = appState.cloneRoleStatusList();
+    List<RoleStatus> roleStatuses =
+        new ArrayList<>(appState.getRoleStatusMap().values());
     Collections.sort(roleStatuses, new RoleStatus.CompareByName());
     for (RoleStatus status : roleStatuses) {
       String roleName = status.getName();
@@ -144,7 +140,7 @@ public class IndexBlock extends SliderHamletBlock {
       String aatext;
       if (status.isAntiAffinePlacement()) {
         boolean aaRequestOutstanding = status.isAARequestOutstanding();
-        int pending = (int)status.getPendingAntiAffineRequests();
+        int pending = (int)status.getAAPending();
         aatext = buildAADetails(aaRequestOutstanding, pending);
         if (SliderUtils.isSet(status.getLabelExpression())) {
           aatext += " (label: " + status.getLabelExpression() + ")";
@@ -160,17 +156,17 @@ public class IndexBlock extends SliderHamletBlock {
         } else {
           aatext = "";
         }
-        if (status.getRequested() > 0) {
+        if (status.getPending() > 0) {
           roleWithOpenRequest ++;
         }
       }
+      SliderMetrics metrics = status.getComponentMetrics();
       table.tr()
         .td().a(nameUrl, roleName)._()
-        .td(String.format("%d", status.getDesired()))
-        .td(String.format("%d", status.getActual()))
-        .td(String.format("%d", status.getRequested()))
-        .td(String.format("%d", status.getFailed()))
-        .td(String.format("%d", status.getStartFailed()))
+        .td(String.format("%d", metrics.containersDesired.value()))
+        .td(String.format("%d", metrics.containersRunning.value()))
+        .td(String.format("%d", metrics.containersPending.value()))
+        .td(String.format("%d", metrics.containersFailed.value()))
         .td(aatext)
         ._();
     }
@@ -218,7 +214,7 @@ public class IndexBlock extends SliderHamletBlock {
     DIV<Hamlet> provider_info = html.div("provider_info");
     provider_info.h3(providerName + " information");
     UL<Hamlet> ul = html.ul();
-    addProviderServiceOptions(providerService, ul, clusterStatus);
+    //TODO render app/cluster status
     ul._();
     provider_info._();
 
@@ -250,40 +246,9 @@ public class IndexBlock extends SliderHamletBlock {
   }
 
   private String getProviderName() {
-    return providerService.getHumanName();
+    return "docker";
   }
 
-  private String getInfoAvoidingNulls(String key) {
-    String createTime = appState.getClusterStatus().getInfo(key);
-
-    return null == createTime ? "N/A" : createTime;
-  }
-
-  protected void addProviderServiceOptions(ProviderService provider,
-      UL ul, ClusterDescription clusterStatus) {
-    Map<String, MonitorDetail> details = provider.buildMonitorDetails(
-        clusterStatus);
-    if (null == details) {
-      return;
-    }
-    // Loop over each entry, placing the text in the UL, adding an anchor when the URL is non-null/empty
-    for (Entry<String, MonitorDetail> entry : details.entrySet()) {
-      MonitorDetail detail = entry.getValue();
-      if (SliderUtils.isSet(detail.getValue()) ) {
-        LI item = ul.li();
-        item.span().$class("bold")._(entry.getKey())._();
-        item._(" - ");
-        if (detail.isUrl()) {
-          // Render an anchor if the value is a URL
-          item.a(detail.getValue(), detail.getValue())._();
-        } else {
-          item._(detail.getValue())._();
-        }
-      } else {
-        ul.li(entry.getKey());
-      }
-    }
-  }
 
   protected void enumeratePublishedExports(PublishedExportsSet exports, UL<Hamlet> ul) {
     for(String key : exports.keys()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/YarnApplicationProbe.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/YarnApplicationProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/YarnApplicationProbe.java
deleted file mode 100644
index 92df048..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/YarnApplicationProbe.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.servicemonitor;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.slider.client.SliderYarnClientImpl;
-import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * Probe for YARN application
- */
-public class YarnApplicationProbe extends Probe {
-  protected static final Logger log = LoggerFactory.getLogger(
-    YarnApplicationProbe.class);
-
-  /**
-   * Yarn client service
-   */
-  private SliderYarnClientImpl yarnClient;
-  private final String clustername;
-  private final String username;
-
-  public YarnApplicationProbe(String clustername,
-                              SliderYarnClientImpl yarnClient,
-                              String name,
-                              Configuration conf, String username)
-      throws IOException {
-    super("Port probe " + name + " " + clustername,
-          conf);
-    this.clustername = clustername;
-    this.yarnClient = yarnClient;
-    this.username = username;
-  }
-
-
-  @Override
-  public void init() throws IOException {
-   
-    log.info("Checking " + clustername );
-  }
-
-  /**
-   * Try to connect to the (host,port); a failure to connect within
-   * the specified timeout is a failure
-   * @param livePing is the ping live: true for live; false for boot time
-   * @return the outcome
-   */
-  @Override
-  public ProbeStatus ping(boolean livePing) {
-    ProbeStatus status = new ProbeStatus();
-    try {
-      List<ApplicationReport> instances = yarnClient
-          .listDeployedInstances(username, null, clustername);
-      ApplicationReport instance = yarnClient
-          .findClusterInInstanceList(instances, clustername);
-      if (null == instance) {
-        throw UnknownApplicationInstanceException.unknownInstance(clustername);
-      }
-      status.succeed(this);
-    } catch (Exception e) {
-      status.fail(this, e);
-    }
-    return status;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
index 254bf27..6defa2b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
@@ -210,8 +210,6 @@ public class YarnRegistryViewForProviders {
 
   /**
    * Add a service under a path for the current user
-   * @param serviceClass service class to use under ~user
-   * @param serviceName name of the service
    * @param record service record
    * @param deleteTreeFirst perform recursive delete of the path first
    * @return the path the service was created at


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org