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 2015/01/14 17:28:41 UTC

[1/5] incubator-slider git commit: SLIDER-754 all application/ URLs are now instrumented

Repository: incubator-slider
Updated Branches:
  refs/heads/develop 68d57cf2a -> 2522f65b3


SLIDER-754 all application/ URLs are now instrumented


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

Branch: refs/heads/develop
Commit: 2522f65b33a24e702b8466b3989b44cab1162373
Parents: 28d296a
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 14 16:27:48 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 14 16:28:27 2015 +0000

----------------------------------------------------------------------
 .../management/MetricsAndMonitoring.java        |  9 +++
 .../web/rest/AbstractSliderResource.java        | 58 ++++++++++++++++++++
 .../rest/application/ApplicationResource.java   | 22 +++++++-
 3 files changed, 88 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2522f65b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
index c14639d..5b96256 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
@@ -91,4 +91,13 @@ public class MetricsAndMonitoring extends CompositeService {
     return instance;
   }
 
+  /**
+   * Get a specific meter and mark it
+   * @param name name of meter/counter
+   */
+  public void markMeterAndCounter(String name) {
+    MeterAndCounter meter = getOrCreateMeterAndCounter(name);
+    meter.mark();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2522f65b/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 dc07c10..cfddb12 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
@@ -23,6 +23,7 @@ import org.apache.hadoop.registry.client.exceptions.AuthenticationFailedExceptio
 import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,9 +43,11 @@ public abstract class AbstractSliderResource {
   private static final Logger log =
       LoggerFactory.getLogger(AbstractSliderResource.class);
   protected final WebAppApi slider;
+  protected final MetricsAndMonitoring metricsAndMonitoring;
 
   public AbstractSliderResource(WebAppApi slider) {
     this.slider = slider;
+    metricsAndMonitoring = slider.getMetricsAndMonitoring();
   }
 
 
@@ -93,4 +96,59 @@ public abstract class AbstractSliderResource {
       return new WebApplicationException(e);
     }
   }
+
+  /**
+   * Mark an GET operation on a path
+   * @param verb HTTP Verb
+   * @param path path relative to slider API
+   */
+  protected void mark(String verb, String path) {
+    metricsAndMonitoring.markMeterAndCounter(verb + "-" + path);
+  }
+  /**
+   * Mark an GET operation on a path
+   * @param verb HTTP Verb
+   * @param path path relative to slider API
+   */
+  protected void mark(String verb, String path, String subpath) {
+    metricsAndMonitoring.markMeterAndCounter(verb + "-" + path + subpath);
+  }
+
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markGet(String path) {
+    mark("GET", path);
+  }
+
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markGet(String path, String subpath) {
+    mark("GET", path, subpath);
+  }
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markPost(String path, String subpath) {
+    mark("POST", path, subpath);
+  }
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markPut(String path, String subpath) {
+    mark("PUT", path, subpath);
+  }
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markDelete(String path, String subpath) {
+    mark("DELETE", path, subpath);
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2522f65b/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
index 6734f73..d72a486 100644
--- 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
@@ -140,6 +140,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path("/")
   @Produces({APPLICATION_JSON})
   public List<String> getRoot() {
+    markGet(SLIDER_SUBPATH_APPLICATION);
     return ROOT_ENTRIES;
   }
 
@@ -151,6 +152,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(MODEL)
   @Produces({APPLICATION_JSON})
   public List<String> getModel() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL);
     return MODEL_ENTRIES;
   }
 
@@ -158,6 +160,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(MODEL_DESIRED)
   @Produces({APPLICATION_JSON})
   public AggregateConf getModelDesired() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED);
     return lookupAggregateConf(MODEL_DESIRED);
   }
   
@@ -165,6 +168,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(MODEL_DESIRED_APPCONF)
   @Produces({APPLICATION_JSON})
   public ConfTree getModelDesiredAppconf() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED_APPCONF);
     return lookupConfTree(MODEL_DESIRED_APPCONF);
   }
 
@@ -172,6 +176,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(MODEL_DESIRED_RESOURCES)
   @Produces({APPLICATION_JSON})
   public ConfTree getModelDesiredResources() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED_RESOURCES);
     return lookupConfTree(MODEL_DESIRED_RESOURCES);
   }
   
@@ -179,6 +184,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(MODEL_RESOLVED)
   @Produces({APPLICATION_JSON})
   public AggregateConf getModelResolved() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED);
     return lookupAggregateConf(MODEL_RESOLVED);
   }
 
@@ -186,6 +192,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(MODEL_RESOLVED_APPCONF)
   @Produces({APPLICATION_JSON})
   public ConfTree getModelResolvedAppconf() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED_APPCONF);
     return lookupConfTree(MODEL_RESOLVED_APPCONF);
   }
 
@@ -193,6 +200,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(MODEL_RESOLVED_RESOURCES)
   @Produces({APPLICATION_JSON})
   public ConfTree getModelResolvedResources() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED_RESOURCES);
     return lookupConfTree(MODEL_RESOLVED_RESOURCES);
   }
   
@@ -200,6 +208,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(LIVE)
   @Produces({APPLICATION_JSON})
   public List<String> getLive() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE);
     return LIVE_ENTRIES;
   }
 
@@ -207,6 +216,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(LIVE_RESOURCES)
   @Produces({APPLICATION_JSON})
   public Object getLiveResources() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_RESOURCES);
     return lookupConfTree(LIVE_RESOURCES);
   }
   
@@ -214,6 +224,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(LIVE_CONTAINERS)
   @Produces({APPLICATION_JSON})
   public Map<String, SerializedContainerInformation> getLiveContainers() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_CONTAINERS);
     try {
       return (Map<String, SerializedContainerInformation>)cache.lookup(
           LIVE_CONTAINERS);
@@ -227,6 +238,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Produces({APPLICATION_JSON})
   public SerializedContainerInformation getLiveContainer(
       @PathParam("containerId") String containerId) {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_CONTAINERS);
     try {
       RoleInstance id = state.getLiveInstanceByContainerID(containerId);
       return id.serialize();
@@ -241,6 +253,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(LIVE_COMPONENTS)
   @Produces({APPLICATION_JSON})
   public Map<String, SerializedComponentInformation> getLiveComponents() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
     try {
       return (Map<String, SerializedComponentInformation>) cache.lookup(
           LIVE_COMPONENTS);
@@ -254,6 +267,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Produces({APPLICATION_JSON})
   public SerializedComponentInformation getLiveComponent(
       @PathParam("component") String component) {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
     try {
       RoleStatus roleStatus = state.lookupRoleStatus(component);
       SerializedComponentInformation info = roleStatus.serialize();
@@ -315,6 +329,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Produces({APPLICATION_JSON})
   public PingResource actionPingGet(@Context HttpServletRequest request,
       @Context UriInfo uriInfo) {
+    markGet(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
     return new RestActionPing().ping(request, uriInfo, "");
   }
   
@@ -324,6 +339,7 @@ public class ApplicationResource extends AbstractSliderResource {
   public PingResource actionPingPost(@Context HttpServletRequest request,
       @Context UriInfo uriInfo,
       String body) {
+    markPost(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
     return new RestActionPing().ping(request, uriInfo, body);
   }
   
@@ -334,6 +350,7 @@ public class ApplicationResource extends AbstractSliderResource {
   public PingResource actionPingPut(@Context HttpServletRequest request,
       @Context UriInfo uriInfo,
       String body) {
+    markPut(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
     return new RestActionPing().ping(request, uriInfo, body);
   }
   
@@ -343,6 +360,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Produces({APPLICATION_JSON})
   public PingResource actionPingDelete(@Context HttpServletRequest request,
       @Context UriInfo uriInfo) {
+    markDelete(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
     return new RestActionPing().ping(request, uriInfo, "");
   }
   
@@ -351,10 +369,11 @@ public class ApplicationResource extends AbstractSliderResource {
   @Produces({APPLICATION_JSON})
   public Object actionPingHead(@Context HttpServletRequest request,
       @Context UriInfo uriInfo) {
+    mark("HEAD", SLIDER_SUBPATH_APPLICATION, ACTION_PING);
     return new RestActionPing().ping(request, uriInfo, "");
   }
   
-    /* ************************************************************************
+  /* ************************************************************************
   
   ACTION STOP
   
@@ -367,6 +386,7 @@ public class ApplicationResource extends AbstractSliderResource {
   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);
   }
 


[5/5] incubator-slider git commit: SLIDER-754 add metering/counter of REST operations

Posted by st...@apache.org.
SLIDER-754 add metering/counter of REST operations


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

Branch: refs/heads/develop
Commit: 61855abf1d14624041e66f73d7f9c4120edf1c35
Parents: 52e5594
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 14 11:23:33 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 14 16:28:27 2015 +0000

----------------------------------------------------------------------
 .../appmaster/management/MeterAndCounter.java   | 96 ++++++++++++++++++++
 .../management/MetricsAndMonitoring.java        | 32 +++++++
 2 files changed, 128 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/61855abf/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java
new file mode 100644
index 0000000..ed55f73
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java
@@ -0,0 +1,96 @@
+/*
+ * 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.management;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricRegistry;
+
+/**
+ * A combined meter and counter that can be used to measure load.
+ * Hash and equality are derived from the name
+ */
+public class MeterAndCounter {
+
+  /**
+   * suffix for counters: {@value}
+   */
+  public static final String COUNTER = ".counter";
+
+  /**
+   * suffix for meters: {@value}
+   */
+  public static final String METER = ".meter";
+
+  final Meter meter;
+  final Counter counter;
+  final String name;
+
+  /**
+   * Construct an instance
+   * @param metrics metrics to bond to
+   * @param name name before suffixes are appended
+   */
+  public MeterAndCounter(MetricRegistry metrics, String name) {
+    this.name = name;
+    counter = metrics.counter(name + COUNTER);
+    meter = metrics.meter(name + METER);
+  }
+
+  /**
+   * Construct an instance
+   * @param metrics metrics to bond to
+   * @param clazz class to use to derive name
+   * @param name name before suffixes are appended
+   */
+
+  public MeterAndCounter(MetricRegistry metrics, Class clazz, String name) {
+    this.name = name;
+    counter = metrics.counter(MetricRegistry.name(clazz, name + COUNTER));
+    meter = metrics.meter(MetricRegistry.name(clazz, name + METER));
+  }
+
+  /**
+   * Increment the counter, mark the meter
+   */
+  public void mark() {
+    counter.inc();
+    meter.mark();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    MeterAndCounter that = (MeterAndCounter) o;
+
+    return name.equals(that.name);
+
+  }
+
+  @Override
+  public int hashCode() {
+    return name.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/61855abf/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
index 77204d6..c14639d 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
@@ -24,6 +24,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
 /**
  * Class for all metrics and monitoring
  */
@@ -44,6 +48,10 @@ public class MetricsAndMonitoring extends CompositeService {
 
   final HealthCheckRegistry health = new HealthCheckRegistry();
 
+  private final Map<String, MeterAndCounter> meterAndCounterMap
+      = new ConcurrentHashMap<String, MeterAndCounter>();
+  
+  
   public MetricRegistry getMetrics() {
     return metrics;
   }
@@ -57,6 +65,30 @@ public class MetricsAndMonitoring extends CompositeService {
     addService(new MetricsBindingService("MetricsBindingService",
         metrics));
     super.serviceInit(conf);
+  }
 
+  public MeterAndCounter getMeterAndCounter(String name) {
+    return meterAndCounterMap.get(name);
   }
+
+  /**
+   * Get or create the meter/counter pair
+   * @param name name of instance
+   * @return an instance
+   */
+  public MeterAndCounter getOrCreateMeterAndCounter(String name) {
+    MeterAndCounter instance = meterAndCounterMap.get(name);
+    if (instance == null) {
+      synchronized (this) {
+        // check in a sync block
+        instance = meterAndCounterMap.get(name);
+        if (instance == null) {
+          instance = new MeterAndCounter(metrics, name);
+          meterAndCounterMap.put(name, instance);
+        }
+      }
+    }
+    return instance;
+  }
+
 }


[4/5] incubator-slider git commit: SLIDER-715 REST stop action completed with mini and functional test

Posted by st...@apache.org.
SLIDER-715 REST stop action completed with mini and functional test


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

Branch: refs/heads/develop
Commit: 28d296a05fb18ccc9c458d97389a302214e6e161
Parents: 2654946
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 14 16:26:32 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 14 16:28:27 2015 +0000

----------------------------------------------------------------------
 .../application/actions/RestActionStop.java     |  5 ++-
 .../slider/agent/rest/RestTestDelegates.groovy  | 47 ++++++++++++++++++--
 .../slider/agent/rest/TestStandaloneREST.groovy |  4 +-
 .../apache/slider/test/SliderTestUtils.groovy   |  2 +-
 .../funtest/lifecycle/AgentWebPagesIT.groovy    | 13 +++---
 5 files changed, 55 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/28d296a0/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
index 703c1e7..f94c983 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
@@ -54,12 +54,13 @@ public class RestActionStop {
     response.text = text;
     ActionStopSlider stopSlider =
         new ActionStopSlider(text,
-            1000, TimeUnit.MILLISECONDS,
+            500,
+            TimeUnit.MILLISECONDS,
             LauncherExitCodes.EXIT_SUCCESS,
             FinalApplicationStatus.SUCCEEDED,
             text);
     log.info("SliderAppMasterApi.stopCluster: {}", stopSlider);
-//    slider.schedule(stopSlider);
+    slider.getQueues().schedule(stopSlider);
     
     return response;
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/28d296a0/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
index 5383425..3431175 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
@@ -32,8 +32,8 @@ import org.apache.slider.core.restclient.HttpVerb
 import org.apache.slider.core.restclient.UrlConnectionOperations
 import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource
 import org.apache.slider.server.appmaster.web.rest.application.resources.PingResource
+import org.apache.slider.test.Outcome
 import org.apache.slider.test.SliderTestUtils
-import org.junit.Test
 
 import javax.ws.rs.core.MediaType
 
@@ -60,12 +60,18 @@ class RestTestDelegates extends SliderTestUtils {
 
   public void testCodahaleOperations() throws Throwable {
     describe "Codahale operations"
-    // now switch to the Hadoop URL connection, with SPNEGO escalation
     getWebPage(appmaster)
     getWebPage(appmaster, SYSTEM_THREADS)
     getWebPage(appmaster, SYSTEM_HEALTHCHECK)
+    getWebPage(appmaster, SYSTEM_PING)
     getWebPage(appmaster, SYSTEM_METRICS_JSON)
   }
+  
+  public void logCodahaleMetrics() {
+    // query Coda Hale metrics
+    log.info getWebPage(appmaster, SYSTEM_HEALTHCHECK)
+    log.info getWebPage(appmaster, SYSTEM_METRICS)
+  }
 
   public void testLiveResources() throws Throwable {
     describe "Live Resources"
@@ -237,8 +243,6 @@ class RestTestDelegates extends SliderTestUtils {
   public void testStop() {
     String target = appendToURL(appmaster, SLIDER_PATH_APPLICATION, ACTION_STOP)
     describe "Stop URL $target"
-
-
     URL targetUrl = new URL(target)
     def outcome = connectionFactory.execHttpOperation(
         HttpVerb.POST,
@@ -247,8 +251,43 @@ class RestTestDelegates extends SliderTestUtils {
         MediaType.TEXT_PLAIN)
     log.info "Stopped: $outcome"
 
+    // await the shutdown
+    sleep(1000)
+    
+    // now a ping is expected to fail
+    String ping = appendToURL(appmaster, SLIDER_PATH_APPLICATION, ACTION_PING)
+    URL pingUrl = new URL(ping)
 
+    repeatUntilSuccess("probe for missing registry entry",
+        this.&probePingFailing, 30000, 500,
+        [url: ping],
+        true,
+        "AM failed to shut down") {
+      def pinged = fetchType(
+          PingResource,
+          appmaster,
+          ACTION_PING + "?body=hello")
+      fail("AM didn't shut down; Ping GET= $pinged")
+    }
+    
   }
 
+  /**
+   * Probe that spins until the url specified by "url") refuses
+   * connections
+   * @param args argument map
+   * @return the outcome
+   */
+  Outcome probePingFailing(Map args) {
+    String ping = args["url"]
+    URL pingUrl = new URL(ping)
+    try {
+      def response = pingAction(HttpVerb.HEAD, pingUrl, "should not be running")
+      return Outcome.Retry
+    } catch (IOException e) {
+      // expected
+      return Outcome.Success
+    }
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/28d296a0/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
index 7ae8e2f..a3378a8 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
@@ -39,7 +39,7 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
 
 
   @Test
-  public void testStandaloneAgentWeb() throws Throwable {
+  public void testStandaloneREST() throws Throwable {
 
     describe "create a standalone AM then perform actions on it"
     //launch fake master
@@ -74,7 +74,6 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
       log.info metrics
     }
     
-    sleep(5000)
     def appmaster = report.trackingUrl
 
     GET(appmaster)
@@ -113,6 +112,7 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
       proxied.testRESTModel()
     }
     
+    direct.logCodahaleMetrics();
     direct.testStop();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/28d296a0/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
index 82436e2..2c6b5fe 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
@@ -1189,7 +1189,7 @@ class SliderTestUtils extends Assert {
     if (timeout < 1000) {
       fail("Timeout $timeout too low: milliseconds are expected, not seconds")
     }
-    int attemptCount = 0
+    int attemptCount = 1
     boolean succeeded = false;
     boolean completed = false;
     Duration duration = new Duration(timeout)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/28d296a0/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentWebPagesIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentWebPagesIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentWebPagesIT.groovy
index 9e5e2cd..ce1d955 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentWebPagesIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentWebPagesIT.groovy
@@ -56,7 +56,7 @@ public class AgentWebPagesIT extends AgentCommandTestBase
 
   @Test
   public void testAgentWeb() throws Throwable {
-    describe("Create a 0-role cluster and make web queries against it")
+    describe("Web queries & REST operations against an AM")
     
     // verify the ws/ path is open for all HTTP verbs
     def sliderConfiguration = ConfigHelper.loadSliderConfiguration();
@@ -98,12 +98,6 @@ public class AgentWebPagesIT extends AgentCommandTestBase
     // get the root page, 
     getWebPage(appmaster)
     
-    // query Coda Hale metrics
-    log.info getWebPage(appmaster, RestPaths.SYSTEM_METRICS)
-    log.info getWebPage(appmaster, RestPaths.SYSTEM_THREADS)
-    log.info getWebPage(appmaster, RestPaths.SYSTEM_HEALTHCHECK)
-    log.info getWebPage(appmaster, RestPaths.SYSTEM_PING)
-
     def realappmaster = report.origTrackingUrl;
     // now attempt direct-to-AM pings
     RestTestDelegates proxied = new RestTestDelegates(appmaster)
@@ -121,6 +115,11 @@ public class AgentWebPagesIT extends AgentCommandTestBase
       // and via the proxy
       proxied.testRESTModel()
     }
+    
+    direct.logCodahaleMetrics();
+    
+    // finally, stop the AM
+    direct.testStop();
   }
 
 }


[2/5] incubator-slider git commit: SLIDER-715 REST STOP action...add queues accessor to state access view of REST API

Posted by st...@apache.org.
SLIDER-715 REST STOP action...add queues accessor to state access view of REST API


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

Branch: refs/heads/develop
Commit: 2654946180aa4eb6db0fe095f025f2509c446192
Parents: 61855ab
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 14 13:57:35 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 14 16:28:27 2015 +0000

----------------------------------------------------------------------
 .../slider/server/appmaster/SliderAppMaster.java    |  3 ++-
 .../slider/server/appmaster/state/AppState.java     |  2 +-
 .../appmaster/state/StateAccessForProviders.java    |  5 +++++
 .../slider/server/appmaster/web/WebAppApi.java      | 16 ++++++++++++++++
 .../slider/server/appmaster/web/WebAppApiImpl.java  | 11 ++++++++++-
 .../TestMockAppStateAppRestIntegration.groovy       |  2 +-
 .../web/view/TestClusterSpecificationBlock.groovy   |  2 +-
 .../web/view/TestContainerStatsBlock.groovy         |  2 +-
 .../server/appmaster/web/view/TestIndexBlock.groovy |  2 +-
 .../web/rest/agent/TestAMAgentWebServices.java      |  3 ++-
 .../management/TestAMManagementWebServices.java     |  3 ++-
 11 files changed, 42 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/26549461/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 06e6c1d..2629a4d 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -902,7 +902,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
               providerService,
               certificateManager,
               registryOperations,
-              metricsAndMonitoring);
+              metricsAndMonitoring,
+              actionQueues);
       initAMFilterOptions(serviceConf);
 
       // start the agent web app

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/26549461/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
index f381cde..4b37e14 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
@@ -888,7 +888,7 @@ public class AppState {
   }
 
   /**
-   * Get a clone of the role status list. Concurrent events may mean this
+   * 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
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/26549461/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
index 2cbe6b1..057c07b 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
@@ -214,5 +214,10 @@ public interface StateAccessForProviders {
    */
   void 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();
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/26549461/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
index 6aa24c4..db32e48 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
@@ -19,6 +19,7 @@ package org.apache.slider.server.appmaster.web;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.slider.api.SliderClusterProtocol;
 import org.apache.slider.providers.ProviderService;
+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.RoleStatus;
@@ -66,7 +67,22 @@ public interface WebAppApi {
   AgentRestOperations getAgentRestOperations();
 
 
+  /**
+   * Registry operations accessor
+   * @return registry access
+   */
   RegistryOperations getRegistryOperations();
 
+  /**
+   * Metrics and monitoring service
+   * @return the (singleton) instance
+   */
   MetricsAndMonitoring getMetricsAndMonitoring();
+
+  /**
+   * Get the queue accessor
+   * @return the immediate and scheduled queues
+   */
+  QueueAccess getQueues();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/26549461/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
index d5532ea..0597c39 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
@@ -19,6 +19,7 @@ package org.apache.slider.server.appmaster.web;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.slider.api.SliderClusterProtocol;
 import org.apache.slider.providers.ProviderService;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 import org.apache.slider.server.appmaster.state.RoleStatus;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
@@ -45,16 +46,19 @@ public class WebAppApiImpl implements WebAppApi {
   protected final CertificateManager certificateManager;
   private final RegistryOperations registryOperations;
   private final MetricsAndMonitoring metricsAndMonitoring;
+  private final QueueAccess queues;
 
   public WebAppApiImpl(SliderClusterProtocol clusterProto,
       StateAccessForProviders appState,
       ProviderService provider,
       CertificateManager certificateManager,
       RegistryOperations registryOperations,
-      MetricsAndMonitoring metricsAndMonitoring) {
+      MetricsAndMonitoring metricsAndMonitoring,
+      QueueAccess queues) {
     checkNotNull(clusterProto);
     checkNotNull(appState);
     checkNotNull(provider);
+    this.queues = queues;
 
     this.registryOperations = registryOperations;
     this.clusterProto = clusterProto;
@@ -109,4 +113,9 @@ public class WebAppApiImpl implements WebAppApi {
   public MetricsAndMonitoring getMetricsAndMonitoring() {
     return metricsAndMonitoring;
   }
+
+  @Override
+  public QueueAccess getQueues() {
+    return queues;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/26549461/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy
index cf32f33..4672da9 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy
@@ -118,7 +118,7 @@ class TestMockAppStateAppRestIntegration extends BaseMockAppStateTest implements
         stateAccess,
         new MockProviderService(),
         null, null,
-        new MetricsAndMonitoring("metrics"))
+        new MetricsAndMonitoring("metrics"), null)
     return api
   }
 

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/26549461/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
index 12ca949..4aac2f1 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
@@ -144,7 +144,8 @@ public class TestAMAgentWebServices {
                                                              appState);
 
     slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
-                               new MockProviderService(), null, null, null);
+                               new MockProviderService(), null, null, null,
+        null);
 
     MapOperations compOperations = new MapOperations();
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/26549461/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
index 0776afc..f19f8f3 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
@@ -192,7 +192,8 @@ public class TestAMManagementWebServices extends JerseyTest {
             appState);
 
         slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
-                                   new MockProviderService(), null, null, null);
+                                   new MockProviderService(), null, null, null,
+            null);
 
         bind(SliderJacksonJaxbJsonProvider.class);
         bind(MockSliderAMWebServices.class);


[3/5] incubator-slider git commit: SLIDER-715 stub REST stop action with test

Posted by st...@apache.org.
SLIDER-715 stub REST stop action with test


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

Branch: refs/heads/develop
Commit: 52e559470f082c07acbd132c58a77b2893d63f2e
Parents: 68d57cf
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 14 11:22:49 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 14 16:28:27 2015 +0000

----------------------------------------------------------------------
 .../server/appmaster/web/rest/RestPaths.java    |  1 +
 .../rest/application/ApplicationResource.java   | 33 ++++++++--
 .../application/actions/RestActionPing.java     |  2 +-
 .../application/actions/RestActionStop.java     | 66 ++++++++++++++++++++
 .../rest/application/actions/StopResponse.java  | 29 +++++++++
 .../slider/agent/rest/RestTestDelegates.groovy  | 25 +++++++-
 .../slider/agent/rest/TestStandaloneREST.groovy |  2 +
 7 files changed, 150 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/52e55947/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 58520f5..35f3e13 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
@@ -145,4 +145,5 @@ public class RestPaths {
 
   public static final String ACTION = "/action";
   public static final String ACTION_PING = ACTION + "/ping";
+  public static final String ACTION_STOP = ACTION + "/stop";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/52e55947/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
index af310b1..6734f73 100644
--- 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
@@ -33,6 +33,8 @@ 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.AggregateModelRefresher;
 import org.apache.slider.server.appmaster.web.rest.application.resources.AppconfRefresher;
 import org.apache.slider.server.appmaster.web.rest.application.resources.CachedContent;
@@ -41,6 +43,7 @@ import org.apache.slider.server.appmaster.web.rest.application.resources.Content
 import org.apache.slider.server.appmaster.web.rest.application.resources.LiveComponentsRefresher;
 import org.apache.slider.server.appmaster.web.rest.application.resources.LiveResourcesRefresher;
 import org.apache.slider.server.appmaster.web.rest.application.actions.RestActionPing;
+import org.apache.slider.server.appmaster.web.rest.application.resources.PingResource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -301,10 +304,16 @@ public class ApplicationResource extends AbstractSliderResource {
     }
   }
 
+  /* ************************************************************************
+  
+  ACTION PING
+  
+  **************************************************************************/
+  
   @GET
   @Path(ACTION_PING)
   @Produces({APPLICATION_JSON})
-  public Object actionPingGet(@Context HttpServletRequest request,
+  public PingResource actionPingGet(@Context HttpServletRequest request,
       @Context UriInfo uriInfo) {
     return new RestActionPing().ping(request, uriInfo, "");
   }
@@ -312,7 +321,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @POST
   @Path(ACTION_PING)
   @Produces({APPLICATION_JSON})
-  public Object actionPingPost(@Context HttpServletRequest request,
+  public PingResource actionPingPost(@Context HttpServletRequest request,
       @Context UriInfo uriInfo,
       String body) {
     return new RestActionPing().ping(request, uriInfo, body);
@@ -322,7 +331,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(ACTION_PING)
   @Consumes({TEXT_PLAIN})
   @Produces({APPLICATION_JSON})
-  public Object actionPingPut(@Context HttpServletRequest request,
+  public PingResource actionPingPut(@Context HttpServletRequest request,
       @Context UriInfo uriInfo,
       String body) {
     return new RestActionPing().ping(request, uriInfo, body);
@@ -332,7 +341,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @Path(ACTION_PING)
   @Consumes({APPLICATION_JSON})
   @Produces({APPLICATION_JSON})
-  public Object actionPingDelete(@Context HttpServletRequest request,
+  public PingResource actionPingDelete(@Context HttpServletRequest request,
       @Context UriInfo uriInfo) {
     return new RestActionPing().ping(request, uriInfo, "");
   }
@@ -345,4 +354,20 @@ public class ApplicationResource extends AbstractSliderResource {
     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) {
+    return new RestActionStop(slider).stop(request, uriInfo, body);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/52e55947/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
index 65126ac..f20f296 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
@@ -34,7 +34,7 @@ public class RestActionPing {
   public RestActionPing() {
   }
   
-  public Object ping(HttpServletRequest request, UriInfo uriInfo, String body) {
+  public PingResource ping(HttpServletRequest request, UriInfo uriInfo, String body) {
     String verb = request.getMethod();
     log.info("Ping {}", verb);
     PingResource pingResource = new PingResource();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/52e55947/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
new file mode 100644
index 0000000..703c1e7
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
@@ -0,0 +1,66 @@
+/*
+ * 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.actions;
+
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.slider.core.main.LauncherExitCodes;
+import org.apache.slider.server.appmaster.actions.ActionStopSlider;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.application.resources.PingResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.UriInfo;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+
+public class RestActionStop {
+  private static final Logger log =
+      LoggerFactory.getLogger(RestActionStop.class);
+
+  private final WebAppApi slider;
+  
+  public RestActionStop(WebAppApi slider) {
+    this.slider = slider;
+  }
+  
+  public StopResponse stop(HttpServletRequest request, UriInfo uriInfo, String body) {
+    String verb = request.getMethod();
+    log.info("Ping {}", verb);
+    StopResponse response = new StopResponse();
+    response.verb = verb;
+    long time = System.currentTimeMillis();
+    String text = 
+        String.format(Locale.ENGLISH,
+            "Stopping action %s received at %tc",
+            verb, time);
+    response.text = text;
+    ActionStopSlider stopSlider =
+        new ActionStopSlider(text,
+            1000, TimeUnit.MILLISECONDS,
+            LauncherExitCodes.EXIT_SUCCESS,
+            FinalApplicationStatus.SUCCEEDED,
+            text);
+    log.info("SliderAppMasterApi.stopCluster: {}", stopSlider);
+//    slider.schedule(stopSlider);
+    
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/52e55947/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java
new file mode 100644
index 0000000..9af6a22
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java
@@ -0,0 +1,29 @@
+/*
+ * 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.actions;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class StopResponse {
+  String verb;
+  public String text;
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/52e55947/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
index 430ddf1..5383425 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
@@ -67,7 +67,6 @@ class RestTestDelegates extends SliderTestUtils {
     getWebPage(appmaster, SYSTEM_METRICS_JSON)
   }
 
-  @Test
   public void testLiveResources() throws Throwable {
     describe "Live Resources"
     ConfTreeOperations tree = fetchConfigTree(appmaster, LIVE_RESOURCES)
@@ -84,8 +83,7 @@ class RestTestDelegates extends SliderTestUtils {
     assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_COMPLETED)
     assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_RELEASING)
   }
-
-  @Test
+  
   public void testLiveContainers() throws Throwable {
     describe "Application REST ${LIVE_CONTAINERS}"
 
@@ -231,5 +229,26 @@ class RestTestDelegates extends SliderTestUtils {
     return outcome
   }
 
+  /**
+   * Test the stop command.
+   * Important: once executed, the AM is no longer there.
+   * This must be the last test in the sequence.
+   */
+  public void testStop() {
+    String target = appendToURL(appmaster, SLIDER_PATH_APPLICATION, ACTION_STOP)
+    describe "Stop URL $target"
+
+
+    URL targetUrl = new URL(target)
+    def outcome = connectionFactory.execHttpOperation(
+        HttpVerb.POST,
+        targetUrl,
+        new byte[0],
+        MediaType.TEXT_PLAIN)
+    log.info "Stopped: $outcome"
+
+
+  }
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/52e55947/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
index 4b907e1..7ae8e2f 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
@@ -112,6 +112,8 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
       // and via the proxy
       proxied.testRESTModel()
     }
+    
+    direct.testStop();
   }