You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by el...@apache.org on 2015/01/29 18:23:35 UTC

[12/28] incubator-slider git commit: SLIDER-762 API-level slider REST client. "Yes, but where are the tests"

SLIDER-762 API-level slider REST client. "Yes, but where are the tests"


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

Branch: refs/heads/develop
Commit: 5b5dac889698ecd5428ac1e0b89c271b97f75c4d
Parents: 1282802
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jan 22 18:27:44 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jan 22 18:27:44 2015 +0000

----------------------------------------------------------------------
 .../apache/slider/client/SliderRestClient.java  | 332 +++++++++++++
 .../core/restclient/UgiJerseyBinding.java       |  44 ++
 .../restclient/UrlConnectionOperations.java     |  25 +-
 .../server/appmaster/state/RoleStatus.java      |   5 +
 .../rest/application/ApplicationResource.java   |   2 +-
 .../agent/rest/JerseyTestDelegates.groovy       |  24 +-
 .../rest/SliderRestClientTestDelegates.groovy   | 476 +++++++++++++++++++
 7 files changed, 885 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b5dac88/slider-core/src/main/java/org/apache/slider/client/SliderRestClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderRestClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderRestClient.java
new file mode 100644
index 0000000..14b85e9
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderRestClient.java
@@ -0,0 +1,332 @@
+/*
+ * 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.client;
+
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.GenericType;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.representation.Form;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.slider.api.types.SerializedComponentInformation;
+import org.apache.slider.api.types.SerializedContainerInformation;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.restclient.HttpVerb;
+import org.apache.slider.core.restclient.UgiJerseyBinding;
+import org.apache.slider.server.appmaster.web.rest.application.resources.PingResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.core.MediaType;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
+public class SliderRestClient  extends AbstractService {
+  private static final Logger log =
+      LoggerFactory.getLogger(SliderRestClient.class);
+  private final Client jersey;
+  private WebResource appmaster;
+  private WebResource appResource;
+
+  public SliderRestClient(String name, Client jersey, WebResource appmaster) {
+    super(name);
+    Preconditions.checkNotNull(jersey, "null jersey");
+    this.jersey = jersey;
+  }
+  
+  public SliderRestClient(Client jersey, WebResource appmaster) {
+    this("SliderRestClient", jersey, appmaster);
+  }
+
+  public Client getJersey() {
+    return jersey;
+  }
+
+  public void bindToAppmaster(WebResource appmaster) {
+    this.appmaster = appmaster;
+    this.appResource = appmaster.path(SLIDER_PATH_APPLICATION);
+  }
+
+  public WebResource getAppmaster() {
+    return appmaster;
+  }
+
+  /**
+   * Create a resource under the application path
+   * @param subpath
+   * @return an resource under the application path
+   */
+  public WebResource applicationResource(String subpath) {
+    return appResource.path(subpath);
+  }
+  
+  /**
+   * Get operation against a path under the Application
+   * @param <T> type expected
+   * @param subpath path
+   * @param c class to instantiate
+   * @return instance
+   * @throws IOException on any problem
+   */
+  public <T> T getApplicationResource(String subpath, Class<T> c)
+      throws IOException {
+    return appResourceOperation(HttpVerb.GET, subpath, c);
+  } 
+  
+  /**
+   * Get operation against a path under the Application
+   * @param <T> type expected
+   * @param subpath path
+   * @param t type info
+   * @return instance
+   * @throws IOException on any problem
+   */
+  public <T> T getApplicationResource(String subpath, GenericType<T> t)
+      throws IOException {
+    return appResourceOperation(HttpVerb.GET, subpath, t);
+  }
+
+  /**
+   * 
+   * @param method method to exec
+   * @param <T> type expected
+   * @param subpath path
+   * @param c class to instantiate
+   * @return instance
+   * @throws IOException on any problem
+   */
+  public <T> T appResourceOperation(HttpVerb method, String subpath, Class<T> c)
+      throws IOException {
+    WebResource resource = applicationResource(subpath);
+    return exec(method, resource, c);
+  }
+  
+  
+  /**
+   * Get operation against a path under the Application
+   * @param <T> type expected
+   * @param subpath path
+   * @param t type info
+   * @return instance
+   * @throws IOException on any problem
+   */
+  public <T> T appResourceOperation(HttpVerb method, String subpath,
+      GenericType<T> t)
+      throws IOException {
+    WebResource resource = applicationResource(subpath);
+    return exec(method, resource, t);
+  }
+  
+  /**
+   * Execute the operation. Failures are raised as IOException subclasses
+   * @param method method to execute
+   * @param resource resource to work against
+   * @param c class to build
+   * @param <T> type expected
+   * @return an instance of the type T
+   * @throws IOException on any failure
+   */
+  public <T> T exec(HttpVerb method, WebResource resource, Class<T> c)
+      throws IOException {
+    try {
+      Preconditions.checkArgument(c != null);
+      resource.accept(MediaType.APPLICATION_JSON_TYPE);
+      return (T) resource.method(method.getVerb(), c);
+    } catch (UniformInterfaceException ex) {
+      throw UgiJerseyBinding.uprateFaults(method, resource.getURI().toString(),
+          ex);
+    }
+  }
+  
+  
+  /**
+   * Execute the operation. Failures are raised as IOException subclasses
+   * @param method method to execute
+   * @param resource resource to work against
+   * @param generic type to work with
+   * @param <T> type expected
+   * @return an instance of the type T
+   * @throws IOException on any failure
+   */
+  public <T> T exec(HttpVerb method, WebResource resource, GenericType<T> t)
+      throws IOException {
+    try {
+      Preconditions.checkArgument(t != null);
+      resource.accept(MediaType.APPLICATION_JSON_TYPE);
+      return resource.method(method.getVerb(), t);
+    } catch (UniformInterfaceException ex) {
+      throw UgiJerseyBinding.uprateFaults(method, resource.getURI().toString(),
+          ex);
+    }
+  }
+  
+  
+
+  /**
+   * Get the aggregate desired model
+   * @return the aggregate configuration of what was asked for
+   * —before resolution has taken place
+   * @throws IOException on any failure
+   */
+  public AggregateConf getDesiredModel() throws IOException {
+    return getApplicationResource(MODEL_DESIRED, AggregateConf.class);
+  }
+
+
+  /**
+   * Get the desired application configuration
+   * @return the application configuration asked for
+   * —before resolution has taken place
+   * @throws IOException on any failure
+   */
+  public ConfTreeOperations getDesiredAppconf() throws IOException {
+    ConfTree resource =
+        getApplicationResource(MODEL_DESIRED_APPCONF, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  /**
+   * Get the desired YARN resources
+   * @return the resources asked for
+   * —before resolution has taken place
+   * @throws IOException on any failure
+   */
+  public ConfTreeOperations getDesiredYarnResources() throws IOException {
+    ConfTree resource =
+        getApplicationResource(MODEL_DESIRED_RESOURCES, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  /**
+   * Get the aggregate resolved model
+   * @return the aggregate configuration of what was asked for
+   * —after resolution has taken place
+   * @throws IOException on any failure
+   */
+  public AggregateConf getResolvedModel() throws IOException {
+    return getApplicationResource(MODEL_RESOLVED, AggregateConf.class);
+  }
+
+
+  /**
+   * Get the resolved application configuration
+   * @return the application configuration asked for
+   * —after resolution has taken place
+   * @throws IOException on any failure
+   */
+  public ConfTreeOperations getResolvedAppconf() throws IOException {
+    ConfTree resource =
+        getApplicationResource(MODEL_RESOLVED_APPCONF, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  /**
+   * Get the resolved YARN resources
+   * @return the resources asked for
+   * —after resolution has taken place
+   * @throws IOException on any failure
+   */
+  public ConfTreeOperations getResolvedYarnResources() throws IOException {
+    ConfTree resource =
+        getApplicationResource(MODEL_RESOLVED_RESOURCES, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  /**
+   * Get the live YARN resources
+   * @return the live set of resources in the cluster
+   * @throws IOException on any failure
+   */
+  public ConfTreeOperations getLiveYarnResources() throws IOException {
+    ConfTree resource =
+        getApplicationResource(LIVE_RESOURCES, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  /**
+   * Get a map of live containers [containerId:info]
+   * @return a possibly empty list of serialized containers
+   * @throws IOException on any failure
+   */
+  public Map<String, SerializedContainerInformation> enumContainers() throws
+      IOException {
+    return getApplicationResource(LIVE_RESOURCES,
+        new GenericType<Map<String, SerializedContainerInformation>>() {
+        });
+  }
+
+  /**
+   * Get a container from the container Id
+   * @param containerId YARN container ID
+   * @return the container information
+   * @throws IOException on any failure
+   */
+  public SerializedContainerInformation getContainer( String containerId) throws
+      IOException {
+    return getApplicationResource(LIVE_CONTAINERS + "/" + containerId,
+        SerializedContainerInformation.class);
+  }
+
+  /**
+   * List all components into a map of [name:info]
+   * @return a possibly empty map of components
+   * @throws IOException on any failure
+   */
+  public Map<String, SerializedComponentInformation> enumComponents() throws
+      IOException {
+    return getApplicationResource(LIVE_COMPONENTS,
+        new GenericType<Map<String, SerializedComponentInformation>>() {
+        });
+  }
+
+  /**
+   * Get information about a component
+   * @param componentName name of the component
+   * @return the component details
+   * @throws IOException on any failure
+   */
+  public SerializedComponentInformation getComponent(String componentName) throws
+      IOException {
+    return getApplicationResource(LIVE_COMPONENTS + "/" + componentName,
+        SerializedComponentInformation.class);
+  }
+
+  /**
+   * Ping as a post
+   * @param text text to include
+   * @return the response
+   * @throws IOException on any failure
+   */
+  public PingResource ping(String text) throws IOException {
+    WebResource pingOut = applicationResource(ACTION_PING);
+    pingOut.accept(MediaType.APPLICATION_JSON_TYPE);
+    pingOut.type(MediaType.APPLICATION_JSON_TYPE);
+    Form f = new Form();
+    f.add("text", text);
+    return pingOut.post(PingResource.class, f);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b5dac88/slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java b/slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java
index 6f002ab..f2991d2 100644
--- a/slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java
@@ -19,10 +19,19 @@
 package org.apache.slider.core.restclient;
 
 import com.google.common.base.Preconditions;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
 import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathAccessDeniedException;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.PathNotFoundException;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.HttpURLConnection;
@@ -38,6 +47,8 @@ import java.net.URL;
  */
 public class UgiJerseyBinding implements
     HttpURLConnectionFactory {
+  private static final Logger log =
+      LoggerFactory.getLogger(UgiJerseyBinding.class);
   private final UrlConnectionOperations operations;
   private final URLConnectionClientHandler handler;
 
@@ -93,6 +104,39 @@ public class UgiJerseyBinding implements
     return operations.isUseSpnego();
   }
 
+
+  /**
+   * Uprate error codes 400 and up into faults; 
+   * 404 is converted to a {@link NotFoundException},
+   * 401 to {@link ForbiddenException}
+   *
+   * @param verb HTTP Verb used
+   * @param url URL as string
+   * @param ex exception
+   */
+  public static IOException uprateFaults(HttpVerb verb, String url,
+      UniformInterfaceException ex)
+      throws IOException {
+
+    ClientResponse response = ex.getResponse();
+    int resultCode = response.getStatus();
+    String msg = verb.toString() + " " + url;
+    if (resultCode == 404) {
+      return (IOException) new PathNotFoundException(url).initCause(ex);
+    }
+    if (resultCode == 401) {
+      return (IOException) new PathAccessDeniedException(url).initCause(ex);
+    }
+    // all other error codes
+
+    
+    // get a string respnse
+    String message = msg +
+                     " failed with exit code " + resultCode
+                     + ", message " + ex.toString();
+    log.error(message, ex);
+    return (IOException) new PathIOException(url, message).initCause(ex);
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b5dac88/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java b/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
index 6e29c55..328684e 100644
--- a/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
@@ -160,7 +160,7 @@ public class UrlConnectionOperations extends Configured  {
         conn.disconnect();
       }
     }
-    uprateFaults(HttpVerb.GET, url, resultCode, body);
+    uprateFaults(HttpVerb.GET, url.toString(), resultCode, "", body);
     outcome.responseCode = resultCode;
     outcome.data = body;
     return outcome;
@@ -174,18 +174,18 @@ public class UrlConnectionOperations extends Configured  {
    * @param verb HTTP Verb used
    * @param url URL as string
    * @param resultCode response from the request
-   * @param body optional body of the request
-   * @throws IOException if the result was considered a failure
+   * @param bodyAsString
+   *@param body optional body of the request  @throws IOException if the result was considered a failure
    */
-  public static void uprateFaults(HttpVerb verb, URL url,
-      int resultCode, byte[] body)
+  public static void uprateFaults(HttpVerb verb, String url,
+      int resultCode, String bodyAsString, byte[] body)
       throws IOException {
 
     if (resultCode < 400) {
       //success
       return;
     }
-    String msg = verb.toString() +" "+ url.toString();
+    String msg = verb.toString() +" "+ url;
     if (resultCode == 404) {
       throw new NotFoundException(msg);
     }
@@ -193,11 +193,14 @@ public class UrlConnectionOperations extends Configured  {
       throw new ForbiddenException(msg);
     }
     // all other error codes
-    String bodyAsString;
-    if (body != null && body.length > 0) {
-      bodyAsString = new String(body);
-    } else {
-      bodyAsString = "";
+    
+    // get a string respnse
+    if (bodyAsString == null) {
+      if (body != null && body.length > 0) {
+        bodyAsString = new String(body);
+      } else {
+        bodyAsString = "";
+      }
     }
     String message =  msg +
                      " failed with exit code " + resultCode

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b5dac88/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
index 0ffc309..3edc5f1 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
@@ -285,6 +285,10 @@ public final class RoleStatus implements Cloneable {
     return stats;
   }
 
+  /**
+   * Produced a serialized form which can be served up as JSON
+   * @return a summary of the current role status.
+   */
   public synchronized SerializedComponentInformation serialize() {
     SerializedComponentInformation info = new SerializedComponentInformation();
     info.name = name;
@@ -298,6 +302,7 @@ public final class RoleStatus implements Cloneable {
     info.requested = requested;
     info.placementPolicy = getPlacementPolicy();
     info.failureMessage = failureMessage;
+    info.totalRequested = totalRequested;
     return info;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b5dac88/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 c2c83a0..735fd48 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
@@ -215,7 +215,7 @@ public class ApplicationResource extends AbstractSliderResource {
   @GET
   @Path(LIVE_RESOURCES)
   @Produces({APPLICATION_JSON})
-  public Object getLiveResources() {
+  public ConfTree getLiveResources() {
     markGet(SLIDER_SUBPATH_APPLICATION, LIVE_RESOURCES);
     return lookupConfTree(LIVE_RESOURCES);
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b5dac88/slider-core/src/test/groovy/org/apache/slider/agent/rest/JerseyTestDelegates.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/JerseyTestDelegates.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/JerseyTestDelegates.groovy
index 09788e1..40f86a8 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/JerseyTestDelegates.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/JerseyTestDelegates.groovy
@@ -18,6 +18,7 @@
 
 package org.apache.slider.agent.rest
 
+import com.google.common.base.Preconditions
 import com.sun.jersey.api.client.Client
 import com.sun.jersey.api.client.ClientResponse
 import com.sun.jersey.api.client.UniformInterfaceException
@@ -29,6 +30,7 @@ import org.apache.hadoop.yarn.webapp.NotFoundException
 import org.apache.slider.api.StateValues
 import org.apache.slider.api.types.SerializedComponentInformation
 import org.apache.slider.api.types.SerializedContainerInformation
+import org.apache.slider.common.tools.SliderUtils
 import org.apache.slider.core.conf.AggregateConf
 import org.apache.slider.core.conf.ConfTree
 import org.apache.slider.core.conf.ConfTreeOperations
@@ -80,7 +82,7 @@ class JerseyTestDelegates extends SliderTestUtils {
    * @return
    */
   public <T> T jGetApplicationResource(String subpath, Class<T> c) {
-    return (T)jExec(HttpVerb.GET, subpath, c)
+    return (T)jExec(HttpVerb.GET, subpath, c);
   }
 
   /**
@@ -90,17 +92,17 @@ class JerseyTestDelegates extends SliderTestUtils {
    * @return
    */
   public <T> T jExec(HttpVerb  method, String subpath, Class<T> c) {
-    WebResource resource = applicationResource(subpath)
-    jExec(method, resource, c)
+    WebResource resource = applicationResource(subpath);
+    jExec(method, resource, c);
   }
 
   public <T> T jExec(HttpVerb method, WebResource resource, Class<T> c) {
     try {
-      assert c
-      resource.accept(MediaType.APPLICATION_JSON_TYPE)
-      (T) resource.method(method.verb, c)
+      Preconditions.checkArgument(c != null);
+      resource.accept(MediaType.APPLICATION_JSON_TYPE);
+      (T) resource.method(method.verb, c);
     } catch (UniformInterfaceException ex) {
-      uprateFaults(method, resource, ex)
+      uprateFaults(method, resource, ex);
     }
   }
 
@@ -158,11 +160,11 @@ class JerseyTestDelegates extends SliderTestUtils {
    */
   public WebResource buildResource(String path) {
     assert path
-    String fullpath = appendToURL(appmaster, path)
-    WebResource webResource = jersey.resource(fullpath)
-    webResource.type(MediaType.APPLICATION_JSON)
+    String fullpath = SliderUtils.appendToURL(appmaster, path);
+    WebResource webResource = jersey.resource(fullpath);
+    webResource.type(MediaType.APPLICATION_JSON);
     log.info("HTTP operation against $fullpath");
-    return webResource
+    return webResource;
   }
 
   public void testJerseyGetConftree() throws Throwable {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b5dac88/slider-core/src/test/groovy/org/apache/slider/agent/rest/SliderRestClientTestDelegates.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/SliderRestClientTestDelegates.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/SliderRestClientTestDelegates.groovy
new file mode 100644
index 0000000..a97d136
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/SliderRestClientTestDelegates.groovy
@@ -0,0 +1,476 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.agent.rest
+
+import com.google.common.base.Preconditions
+import com.sun.jersey.api.client.Client
+import com.sun.jersey.api.client.ClientResponse
+import com.sun.jersey.api.client.UniformInterfaceException
+import com.sun.jersey.api.client.WebResource
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.webapp.NotFoundException
+import org.apache.slider.api.StateValues
+import org.apache.slider.api.types.SerializedComponentInformation
+import org.apache.slider.api.types.SerializedContainerInformation
+import org.apache.slider.common.tools.SliderUtils
+import org.apache.slider.core.conf.AggregateConf
+import org.apache.slider.core.conf.ConfTree
+import org.apache.slider.core.conf.ConfTreeOperations
+import org.apache.slider.core.restclient.HttpVerb
+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.SliderTestUtils
+
+import javax.ws.rs.core.MediaType
+
+import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES
+import static org.apache.slider.api.StatusKeys.*
+import static org.apache.slider.common.SliderKeys.COMPONENT_AM
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*
+
+/**
+ * This class contains parts of tests that can be run
+ * against a deployed AM: local or remote.
+ * It uses Jersey ... and must be passed a client that is either secure
+ * or not
+ * 
+ */
+@CompileStatic
+@Slf4j
+class SliderRestClientTestDelegates extends SliderTestUtils {
+  public static final String TEST_GLOBAL_OPTION = "test.global.option"
+  public static final String TEST_GLOBAL_OPTION_PRESENT = "present"
+
+  final String appmaster;
+  final String application;
+  final Client jersey;
+  final WebResource amResource
+  final WebResource appResource
+
+
+  SliderRestClientTestDelegates(String appmaster, Client jersey) {
+    this.jersey = jersey
+    this.appmaster = appmaster
+    application = appendToURL(appmaster, SLIDER_PATH_APPLICATION)
+    amResource = jersey.resource(appmaster)
+    amResource.type(MediaType.APPLICATION_JSON)
+    appResource = amResource.path(SLIDER_PATH_APPLICATION)
+  }
+
+  /**
+   * <T> T get(Class<T> c)
+   * Get operation against a path under the Application
+   * @param subpath path
+   * @return
+   */
+  public <T> T jGetApplicationResource(String subpath, Class<T> c) {
+    return (T)jExec(HttpVerb.GET, subpath, c);
+  }
+
+  /**
+   * <T> T get(Class<T> c)
+   * Get operation against a path under the Application
+   * @param subpath path
+   * @return
+   */
+  public <T> T jExec(HttpVerb  method, String subpath, Class<T> c) {
+    WebResource resource = applicationResource(subpath);
+    jExec(method, resource, c);
+  }
+
+  public <T> T jExec(HttpVerb method, WebResource resource, Class<T> c) {
+    try {
+      Preconditions.checkArgument(c != null);
+      resource.accept(MediaType.APPLICATION_JSON_TYPE);
+      (T) resource.method(method.verb, c);
+    } catch (UniformInterfaceException ex) {
+      uprateFaults(method, resource, ex);
+    }
+  }
+
+  /**
+   * Create a resource under the application path
+   * @param subpath
+   * @return
+   */
+  public WebResource applicationResource(String subpath) {
+    return appResource.path(subpath)
+  }
+
+  /**
+   * Convert faults to exceptions; pass through 200 responses
+   * @param method
+   * @param webResource
+   * @param ex
+   * @return
+   */
+  public uprateFaults(
+      HttpVerb method,
+      WebResource webResource,
+      UniformInterfaceException ex) {
+    uprateFaults(method.verb,
+        webResource.URI.toString(),
+        ex.response.status,
+        ex.response.toString())
+  }
+
+  /**
+   * <T> T get(Class<T> c)
+   * Get operation against a path under the AM
+   * @param path path
+   * @return
+   */
+  public <T> T jGetAMResource(String path, Class<T> c) {
+    assert c
+    WebResource webResource = buildResource(path)
+    (T)webResource.get(c)
+  }
+
+  /**
+   * Get operation against a path under the AM
+   * @param path path
+   * @return the string value
+   */
+  public String jerseyGet(String path) {
+    return jGetAMResource(path, String.class)
+  }
+
+  /**
+   * Build a resource against a path under the AM API
+   * @param path path
+   * @return a resource for use
+   */
+  public WebResource buildResource(String path) {
+    assert path
+    String fullpath = SliderUtils.appendToURL(appmaster, path);
+    WebResource webResource = jersey.resource(fullpath);
+    webResource.type(MediaType.APPLICATION_JSON);
+    log.info("HTTP operation against $fullpath");
+    return webResource;
+  }
+
+  public void testJerseyGetConftree() throws Throwable {
+    jGetApplicationResource(LIVE_RESOURCES, ConfTree.class);
+  }
+  public void testCodahaleOperations() throws Throwable {
+    describe "Codahale operations"
+    
+    jerseyGet("/")
+    jerseyGet(SYSTEM_THREADS)
+    jerseyGet(SYSTEM_HEALTHCHECK)
+    jerseyGet(SYSTEM_PING)
+    jerseyGet(SYSTEM_METRICS_JSON)
+  }
+  
+  public void logCodahaleMetrics() {
+    // query Coda Hale metrics
+    log.info jerseyGet(SYSTEM_HEALTHCHECK)
+    log.info jerseyGet(SYSTEM_METRICS)
+  }
+
+  /**
+   * Fetch a typed entry <i>under the application path</i>
+   * @param subpath
+   * @param clazz
+   * @return
+   */
+  public <T> T jFetchType(
+      String subpath, Class<T> clazz) {
+    (T)jGetApplicationResource(subpath, clazz)
+  }
+
+  public ConfTreeOperations jGetConfigTree(
+      String path) {
+    ConfTree ctree = jGetApplicationResource(path, ConfTree)
+    ConfTreeOperations tree = new ConfTreeOperations(ctree)
+    return tree
+  }
+
+
+  public void testMimeTypes() throws Throwable {
+    describe "Mime Types"
+
+    WebResource resource = applicationResource(LIVE_RESOURCES)
+    def response = resource.get(ClientResponse)
+    response.headers.each {key, val -> log.info("$key: $val")}
+    log.info response.toString()
+    assert response.type.equals(MediaType.APPLICATION_JSON_TYPE)
+  }
+  
+  
+  public void testLiveResources() throws Throwable {
+    describe "Live Resources"
+
+    ConfTreeOperations tree = jGetConfigTree(LIVE_RESOURCES)
+
+    log.info tree.toString()
+    def liveAM = tree.getComponent(COMPONENT_AM)
+    def desiredInstances = liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES);
+    assert desiredInstances ==
+           liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_ACTUAL)
+
+    assert 1 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_STARTED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_REQUESTING)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_FAILED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_COMPLETED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_RELEASING)
+  }
+
+  public void testLiveContainers() throws Throwable {
+    describe "Application REST ${LIVE_CONTAINERS}"
+
+    Map<String, SerializedContainerInformation> containers =
+        jGetApplicationResource(LIVE_CONTAINERS, HashMap)
+    assert containers.size() == 1
+    log.info "${containers}"
+    SerializedContainerInformation amContainerInfo =
+        (SerializedContainerInformation) containers.values()[0]
+    assert amContainerInfo.containerId
+
+    def amContainerId = amContainerInfo.containerId
+    assert containers[amContainerId]
+
+    assert amContainerInfo.component == COMPONENT_AM
+    assert amContainerInfo.createTime > 0
+    assert amContainerInfo.exitCode == null
+    assert amContainerInfo.output == null
+    assert amContainerInfo.released == null
+    assert amContainerInfo.state == StateValues.STATE_LIVE
+
+    describe "containers"
+
+    SerializedContainerInformation retrievedContainerInfo =
+        jFetchType(
+            LIVE_CONTAINERS + "/${amContainerId}",
+            SerializedContainerInformation
+        )
+    assert retrievedContainerInfo.containerId == amContainerId
+
+    // fetch missing
+    try {
+      def result = jFetchType(
+          LIVE_CONTAINERS + "/unknown",
+          SerializedContainerInformation
+      )
+      fail("expected an error, got $result")
+    } catch (NotFoundException e) {
+      // expected
+    }
+
+
+    describe "components"
+
+    Map<String, SerializedComponentInformation> components =
+        jFetchType(LIVE_COMPONENTS, HashMap)
+    // two components
+    assert components.size() >= 1
+    log.info "${components}"
+
+    SerializedComponentInformation amComponentInfo =
+        (SerializedComponentInformation) components[COMPONENT_AM]
+
+    SerializedComponentInformation amFullInfo = jFetchType(
+        LIVE_COMPONENTS + "/${COMPONENT_AM}",
+        SerializedComponentInformation
+    )
+
+    assert amFullInfo.containers.size() == 1
+    assert amFullInfo.containers[0] == amContainerId
+
+  }
+
+  /**
+   * Assert that a path resolves to an array list that contains
+   * those entries (and only those entries) expected
+   * @param appmaster AM ref
+   * @param path path under AM
+   * @param entries entries to assert the presence of
+   */
+  public void assertPathServesList(
+      String appmaster,
+      String path,
+      List<String> entries) {
+    def list = jFetchType(path, ArrayList)
+    assert list.size() == entries.size()
+    assert entries.containsAll(list)
+  }
+
+  /**
+   * Fetch a list of URLs, all of which must be of the same type
+   * @param clazz class of resolved values
+   * @param appmaster URL to app master
+   * @param subpaths list of subpaths
+   * @return a map of paths to values
+   */
+  public <T> Map<String, T> fetchTypeList(
+      Class<T> clazz, String appmaster, List<String> subpaths
+                                         ) {
+    Map<String, T> results = [:]
+    subpaths.each { String it ->
+      results[it] = (jFetchType(it, clazz))
+    }
+    return results;
+  }
+
+  /**
+   * Test the rest model. For this to work the cluster has to be configured
+   * with the global option
+   * @param appmaster
+   */
+  public void testRESTModel() {
+    describe "model"
+
+    assertPathServesList(appmaster,
+        MODEL,
+        ApplicationResource.MODEL_ENTRIES)
+
+    def unresolvedConf = jFetchType(MODEL_DESIRED, AggregateConf)
+//    log.info "Unresolved \n$unresolvedConf"
+    def unresolvedAppConf = unresolvedConf.appConfOperations
+
+    def sam = "slider-appmaster"
+    assert unresolvedAppConf.getComponentOpt(sam,
+        TEST_GLOBAL_OPTION, "") == ""
+    def resolvedConf = jFetchType(MODEL_RESOLVED, AggregateConf)
+//    log.info "Resolved \n$resolvedConf"
+    assert resolvedConf.appConfOperations.getComponentOpt(
+        sam, TEST_GLOBAL_OPTION, "") == TEST_GLOBAL_OPTION_PRESENT
+
+    def unresolved = fetchTypeList(ConfTree, appmaster,
+        [MODEL_DESIRED_APPCONF, MODEL_DESIRED_RESOURCES])
+    assert unresolved[MODEL_DESIRED_APPCONF].components[sam]
+    [TEST_GLOBAL_OPTION] == null
+
+
+    def resolved = fetchTypeList(ConfTree, appmaster,
+        [MODEL_RESOLVED_APPCONF, MODEL_RESOLVED_RESOURCES])
+    assert resolved[MODEL_RESOLVED_APPCONF].
+               components[sam][TEST_GLOBAL_OPTION] == TEST_GLOBAL_OPTION_PRESENT
+  }
+
+  public void testPing() {
+    // GET
+    describe "pinging"
+    
+    def pinged = jExec(HttpVerb.GET, ACTION_PING, PingResource)
+    log.info "Ping GET: $pinged"
+    // HEAD
+//    jExec(HttpVerb.HEAD, ACTION_PING, PingResource)
+    jExec(HttpVerb.PUT, ACTION_PING, PingResource)
+    jExec(HttpVerb.DELETE, ACTION_PING, PingResource)
+    jExec(HttpVerb.POST, ACTION_PING, PingResource)
+    ping(HttpVerb.PUT, ACTION_PING, "ping-text")
+    ping(HttpVerb.POST, ACTION_PING, "ping-text")
+    ping(HttpVerb.DELETE, ACTION_PING, "ping-text")
+  }
+
+  /**
+   * Execute a ping; assert that a response came back with the relevant
+   * verb if the verb has a response body
+   * @param method method to invoke
+   * @param subpath ping path
+   * @param payload payload
+   * @return the resource if the verb has a response
+   */
+  private PingResource ping(HttpVerb method, String subpath, Object payload) {
+    def actionPing = applicationResource(ACTION_PING)
+    def upload = method.hasUploadBody() ? payload : null
+    if (method.hasResponseBody()) {
+      def pinged = actionPing.method(method.verb, PingResource, upload)
+      assert method.verb == pinged.verb
+      return pinged
+    } else {
+      actionPing.method(method.verb, upload)
+      return null
+    }
+  }
+
+  /**
+   * 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 = connectionOperations.execHttpOperation(
+        HttpVerb.POST,
+        targetUrl,
+        new byte[0],
+        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 = jFetchType(ACTION_PING + "?body=hello",
+          PingResource
+      )
+      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
+    }
+  }
+*/
+
+  public void testSuiteGetOperations() {
+
+    testCodahaleOperations()
+    testMimeTypes()
+    testLiveResources()
+    testLiveContainers();
+    testRESTModel()
+  }
+
+  public void testSuiteComplexVerbs() {
+    testPing();
+  }
+}