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/23 20:43:18 UTC

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

Repository: incubator-slider
Updated Branches:
  refs/heads/develop ea6eca122 -> 6bd71659e


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();
+  }
+}


[4/7] incubator-slider git commit: SLIDER-782 funtest for slider application API

Posted by st...@apache.org.
SLIDER-782 funtest for slider application 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/73e16399
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/73e16399
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/73e16399

Branch: refs/heads/develop
Commit: 73e163998596a9f3acc2ea8f740054c2a9b3434c
Parents: b0963bd
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 23 12:45:52 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 23 12:45:52 2015 +0000

----------------------------------------------------------------------
 .../funtest/lifecycle/AgentWebPagesIT.groovy      | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/73e16399/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 2d1c863..fd54b55 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
@@ -24,6 +24,7 @@ import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.webapp.ForbiddenException
 import org.apache.slider.agent.rest.JerseyTestDelegates
 import org.apache.slider.agent.rest.RestTestDelegates
+import org.apache.slider.agent.rest.SliderRestClientTestDelegates
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
@@ -126,7 +127,22 @@ public class AgentWebPagesIT extends AgentCommandTestBase
         new JerseyTestDelegates(directAM, createUGIJerseyClient())
     directJerseyTests.testSuiteGetOperations()
     directJerseyTests.testSuiteComplexVerbs()
-    
+
+    describe "Proxy SliderRestClient Tests"
+    SliderRestClientTestDelegates proxySliderRestClient =
+        new SliderRestClientTestDelegates(proxyAM, createUGIJerseyClient())
+    proxySliderRestClient.testSuiteGetOperations()
+    if (!wsBackDoorRequired) {
+      proxySliderRestClient.testSuiteComplexVerbs()
+    }
+    describe "Direct SliderRestClient Tests"
+    SliderRestClientTestDelegates directSliderRestClient =
+        new SliderRestClientTestDelegates(directAM, createUGIJerseyClient())
+    directSliderRestClient.testSuiteGetOperations()
+    directSliderRestClient.testSuiteComplexVerbs()
+
+
+
     if (UserGroupInformation.securityEnabled) {
       describe "Insecure Proxy Tests against a secure cluster"
 


[2/7] incubator-slider git commit: SLIDER-762 API-level REST client implemented as far as API goes today

Posted by st...@apache.org.
SLIDER-762 API-level REST client implemented as far as API goes today


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

Branch: refs/heads/develop
Commit: b7d5099d3ab2ddfb54a97ec0b70dc7c4ac7e8ed9
Parents: 5b5dac8
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jan 22 19:11:49 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jan 22 19:11:49 2015 +0000

----------------------------------------------------------------------
 .../apache/slider/client/SliderRestClient.java  |  13 +-
 .../core/restclient/UgiJerseyBinding.java       |   3 +
 .../rest/SliderRestClientTestDelegates.groovy   | 289 ++-----------------
 .../slider/agent/rest/TestStandaloneREST.groovy |  14 +
 4 files changed, 61 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b7d5099d/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
index 14b85e9..9de2551 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderRestClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderRestClient.java
@@ -24,6 +24,7 @@ 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.commons.lang.StringUtils;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.slider.api.types.SerializedComponentInformation;
 import org.apache.slider.api.types.SerializedContainerInformation;
@@ -55,6 +56,9 @@ public class SliderRestClient  extends AbstractService {
     super(name);
     Preconditions.checkNotNull(jersey, "null jersey");
     this.jersey = jersey;
+    if (appmaster !=null) {
+      bindToAppmaster(appmaster);
+    }
   }
   
   public SliderRestClient(Client jersey, WebResource appmaster) {
@@ -65,6 +69,10 @@ public class SliderRestClient  extends AbstractService {
     return jersey;
   }
 
+  /**
+   * Bind/rebind to the AM
+   * @param appmaster AM
+   */
   public void bindToAppmaster(WebResource appmaster) {
     this.appmaster = appmaster;
     this.appResource = appmaster.path(SLIDER_PATH_APPLICATION);
@@ -80,6 +88,9 @@ public class SliderRestClient  extends AbstractService {
    * @return an resource under the application path
    */
   public WebResource applicationResource(String subpath) {
+    Preconditions.checkArgument(!StringUtils.isEmpty(subpath),
+        "empty path");
+    Preconditions.checkNotNull(appResource, "Null app resource");
     return appResource.path(subpath);
   }
   
@@ -273,7 +284,7 @@ public class SliderRestClient  extends AbstractService {
    */
   public Map<String, SerializedContainerInformation> enumContainers() throws
       IOException {
-    return getApplicationResource(LIVE_RESOURCES,
+    return getApplicationResource(LIVE_CONTAINERS,
         new GenericType<Map<String, SerializedContainerInformation>>() {
         });
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b7d5099d/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 f2991d2..76407c6 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
@@ -113,6 +113,9 @@ public class UgiJerseyBinding implements
    * @param verb HTTP Verb used
    * @param url URL as string
    * @param ex exception
+   * @throws PathNotFoundException for an unknown resource
+   * @throws PathAccessDeniedException for access denied
+   * @throws PathIOException for anything else
    */
   public static IOException uprateFaults(HttpVerb verb, String url,
       UniformInterfaceException ex)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b7d5099d/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
index a97d136..46dba1d 100644
--- 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
@@ -18,24 +18,18 @@
 
 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.hadoop.fs.PathNotFoundException
 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.client.SliderRestClient
 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
@@ -61,164 +55,36 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
   final String appmaster;
   final String application;
   final Client jersey;
-  final WebResource amResource
-  final WebResource appResource
+  final SliderRestClient restClient;
 
 
   SliderRestClientTestDelegates(String appmaster, Client jersey) {
     this.jersey = jersey
     this.appmaster = appmaster
     application = appendToURL(appmaster, SLIDER_PATH_APPLICATION)
-    amResource = jersey.resource(appmaster)
+    WebResource amResource = jersey.resource(appmaster)
     amResource.type(MediaType.APPLICATION_JSON)
-    appResource = amResource.path(SLIDER_PATH_APPLICATION)
+    restClient = new SliderRestClient(jersey, amResource)
   }
 
-  /**
-   * <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)
+  public void testGetDesiredModel() throws Throwable {
+      restClient.getDesiredModel()  
+      restClient.getDesiredAppconf()  
+      restClient.getDesiredYarnResources()  
   }
 
-  /**
-   * 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())
+  public void testGetResolvedModel() throws Throwable {
+      restClient.getResolvedModel()  
+      restClient.getResolvedAppconf()  
+      restClient.getResolvedYarnResources()  
   }
 
-  /**
-   * <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)
+    ConfTreeOperations tree = restClient.getLiveYarnResources()
 
     log.info tree.toString()
     def liveAM = tree.getComponent(COMPONENT_AM)
@@ -236,8 +102,7 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
   public void testLiveContainers() throws Throwable {
     describe "Application REST ${LIVE_CONTAINERS}"
 
-    Map<String, SerializedContainerInformation> containers =
-        jGetApplicationResource(LIVE_CONTAINERS, HashMap)
+    Map<String, SerializedContainerInformation> containers = restClient.enumContainers()
     assert containers.size() == 1
     log.info "${containers}"
     SerializedContainerInformation amContainerInfo =
@@ -256,21 +121,15 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
 
     describe "containers"
 
-    SerializedContainerInformation retrievedContainerInfo =
-        jFetchType(
-            LIVE_CONTAINERS + "/${amContainerId}",
-            SerializedContainerInformation
-        )
-    assert retrievedContainerInfo.containerId == amContainerId
+    SerializedContainerInformation amContainerInfo2 =
+        restClient.getContainer(amContainerId)
+    assert amContainerInfo2.containerId == amContainerId
 
     // fetch missing
     try {
-      def result = jFetchType(
-          LIVE_CONTAINERS + "/unknown",
-          SerializedContainerInformation
-      )
+      def result = restClient.getContainer("/unknown")
       fail("expected an error, got $result")
-    } catch (NotFoundException e) {
+    } catch (PathNotFoundException e) {
       // expected
     }
 
@@ -278,7 +137,8 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     describe "components"
 
     Map<String, SerializedComponentInformation> components =
-        jFetchType(LIVE_COMPONENTS, HashMap)
+        restClient.enumComponents()
+
     // two components
     assert components.size() >= 1
     log.info "${components}"
@@ -286,49 +146,14 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     SerializedComponentInformation amComponentInfo =
         (SerializedComponentInformation) components[COMPONENT_AM]
 
-    SerializedComponentInformation amFullInfo = jFetchType(
-        LIVE_COMPONENTS + "/${COMPONENT_AM}",
-        SerializedComponentInformation
-    )
+    SerializedComponentInformation amFullInfo = restClient.getComponent(COMPONENT_AM) 
 
     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
@@ -341,15 +166,14 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
         MODEL,
         ApplicationResource.MODEL_ENTRIES)
 
-    def unresolvedConf = jFetchType(MODEL_DESIRED, AggregateConf)
+    def unresolvedConf = restClient.getDesiredModel() 
 //    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"
+    def resolvedConf = restClient.getResolvedModel() 
     assert resolvedConf.appConfOperations.getComponentOpt(
         sam, TEST_GLOBAL_OPTION, "") == TEST_GLOBAL_OPTION_PRESENT
 
@@ -359,9 +183,9 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     [TEST_GLOBAL_OPTION] == null
 
 
-    def resolved = fetchTypeList(ConfTree, appmaster,
-        [MODEL_RESOLVED_APPCONF, MODEL_RESOLVED_RESOURCES])
-    assert resolved[MODEL_RESOLVED_APPCONF].
+    
+    def resolvedAppconf = restClient.getResolvedAppconf() 
+    assert resolvedAppconf.
                components[sam][TEST_GLOBAL_OPTION] == TEST_GLOBAL_OPTION_PRESENT
   }
 
@@ -369,38 +193,9 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     // 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")
+    restClient.ping("hello")
   }
 
-  /**
-   * 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.
@@ -441,30 +236,10 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
   }
 */
 
-  /**
-   * 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()
+    testGetDesiredModel()
+    testGetResolvedModel()
     testLiveResources()
     testLiveContainers();
     testRESTModel()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b7d5099d/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 0abf264..8ee88c3 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
@@ -110,6 +110,20 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
 
     
     
+    
+    describe "Proxy SliderRestClient Tests"
+    SliderRestClientTestDelegates proxySliderRestClient =
+        new SliderRestClientTestDelegates(proxyAM, createUGIJerseyClient())
+    proxySliderRestClient.testSuiteGetOperations()
+
+    describe "Direct SliderRestClient Tests"
+    SliderRestClientTestDelegates directSliderRestClient =
+        new SliderRestClientTestDelegates(directAM, createUGIJerseyClient())
+    directSliderRestClient.testSuiteGetOperations()
+    directSliderRestClient.testSuiteComplexVerbs()
+
+    
+    
     describe "Proxy Jersey Tests"
     JerseyTestDelegates proxyJerseyTests =
         new JerseyTestDelegates(proxyAM, createUGIJerseyClient())


[5/7] incubator-slider git commit: SLIDER-782 Failed attempt to integrate with the AmWebClient logic for handling redirects across http/https boundary. That code only works for GET operations.

Posted by st...@apache.org.
SLIDER-782 Failed attempt to integrate with the AmWebClient logic for handling redirects across http/https boundary. That code only works for GET 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/6c038b41
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/6c038b41
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/6c038b41

Branch: refs/heads/develop
Commit: 6c038b4132fe8bfdca493d3ef9158bfd838db9db
Parents: 73e1639
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 23 12:46:34 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 23 12:48:16 2015 +0000

----------------------------------------------------------------------
 .../core/restclient/UgiJerseyBinding.java       | 12 +++-
 .../restclient/UrlConnectionOperations.java     | 69 ++++++++++++++++++--
 2 files changed, 73 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6c038b41/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 76407c6..10ce1ce 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
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.servlet.http.HttpServletResponse;
 import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.URL;
@@ -74,7 +75,7 @@ public class UgiJerseyBinding implements
 
   /**
    * Get a URL connection. 
-   * @param url
+   * @param url URL to connect to
    * @return the connection
    * @throws IOException any problem. {@link AuthenticationException} 
    * errors are wrapped
@@ -82,6 +83,10 @@ public class UgiJerseyBinding implements
   @Override
   public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
     try {
+      // open a connection handling status codes and so redirections
+      // but as it opens a connection, it's less useful than you think.
+//      return operations.openConnectionRedirecting(url);
+      
       return operations.openConnection(url);
     } catch (AuthenticationException e) {
       throw new IOException(e);
@@ -124,10 +129,11 @@ public class UgiJerseyBinding implements
     ClientResponse response = ex.getResponse();
     int resultCode = response.getStatus();
     String msg = verb.toString() + " " + url;
-    if (resultCode == 404) {
+    if (resultCode == HttpServletResponse.SC_NOT_FOUND) {
       return (IOException) new PathNotFoundException(url).initCause(ex);
     }
-    if (resultCode == 401) {
+    if (resultCode == HttpServletResponse.SC_UNAUTHORIZED
+        || resultCode == HttpServletResponse.SC_FORBIDDEN) {
       return (IOException) new PathAccessDeniedException(url).initCause(ex);
     }
     // all other error codes

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6c038b41/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 328684e..e6b08c1 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
@@ -31,10 +31,13 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.HttpHeaders;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.HttpURLConnection;
+import java.net.URI;
 import java.net.URL;
 
 /**
@@ -74,7 +77,8 @@ public class UrlConnectionOperations extends Configured  {
   }
 
   /**
-   * Opens a url with read and connect timeouts
+   * Opens a url with cache disabled, redirect handled in 
+   * (JDK) implementation.
    *
    * @param url to open
    * @return URLConnection
@@ -84,14 +88,70 @@ public class UrlConnectionOperations extends Configured  {
   public HttpURLConnection openConnection(URL url) throws
       IOException,
       AuthenticationException {
-    Preconditions.checkArgument(url.getPort() != 0, "no port");
-    HttpURLConnection conn =
-        (HttpURLConnection) connectionFactory.openConnection(url, useSpnego);
+
+    HttpURLConnection conn = innerOpenConnection(url);
     conn.setUseCaches(false);
     conn.setInstanceFollowRedirects(true);
     return conn;
   }
 
+
+  /**
+   * Opens a url.
+   * <p>
+   *   This implementation
+   *   <ol>
+   *     <li>Handles protocol switching during redirects</li>
+   *     <li>Handles 307 responses "redirect with same verb"</li>
+   *   </ol>
+   *
+   * @param url to open
+   * @return URLConnection
+   * @throws IOException
+   * @throws AuthenticationException authentication failure
+   */
+  public HttpURLConnection openConnectionRedirecting(URL url) throws
+      IOException,
+      AuthenticationException {
+    HttpURLConnection connection = innerOpenConnection(url);
+    connection.setUseCaches(false);
+    int responseCode = connection.getResponseCode();
+    if (responseCode == HttpServletResponse.SC_MOVED_TEMPORARILY 
+        || responseCode == HttpServletResponse.SC_TEMPORARY_REDIRECT) {
+      log.debug("Redirected with response {}", responseCode);
+      // is a redirect - are we changing schemes?
+      String redirectLocation = connection.getHeaderField(HttpHeaders.LOCATION);
+      String originalScheme = url.getProtocol();
+      String redirectScheme = URI.create(redirectLocation).getScheme();
+      boolean buildNewUrl = false;
+      if (!originalScheme.equals(redirectScheme)) {
+        // need to fake it out by doing redirect ourselves
+        log.debug("Protocol change during redirect");
+        buildNewUrl = true;
+      } else if (responseCode == HttpServletResponse.SC_TEMPORARY_REDIRECT) {
+        // 307 response
+        buildNewUrl = true;
+      }
+
+      if (buildNewUrl) {
+        // perform redirect ourselves
+        log.debug("Redirecting {} to URL {}",
+            url, redirectLocation);
+        URL redirectURL = new URL(redirectLocation);
+        connection = innerOpenConnection(url);
+      }
+    }
+
+    return connection;
+  }
+
+  protected HttpURLConnection innerOpenConnection(URL url) throws
+      IOException,
+      AuthenticationException {
+    Preconditions.checkArgument(url.getPort() != 0, "no port");
+    return (HttpURLConnection) connectionFactory.openConnection(url, useSpnego);
+  }
+
   public HttpOperationResponse execGet(URL url) throws
       IOException,
       AuthenticationException {
@@ -121,7 +181,6 @@ public class UrlConnectionOperations extends Configured  {
       if (doOutput) {
         conn.setRequestProperty("Content-Type", contentType);
       }
-      
 
       // now do the connection
       conn.connect();


[3/7] incubator-slider git commit: SLIDER-782 factoring out base for other jersey clients;

Posted by st...@apache.org.
SLIDER-782 factoring out base for other jersey clients;


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

Branch: refs/heads/develop
Commit: b0963bd02a3cd1994848facec16c2076d79f9a38
Parents: b7d5099
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 23 12:45:22 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 23 12:45:22 2015 +0000

----------------------------------------------------------------------
 .../apache/slider/client/SliderRestClient.java  | 343 -------------------
 .../slider/client/rest/BaseRestClient.java      | 154 +++++++++
 .../client/rest/SliderApplicationAPI.java       | 274 +++++++++++++++
 .../rest/SliderRestClientTestDelegates.groovy   |  38 +-
 4 files changed, 447 insertions(+), 362 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b0963bd0/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
deleted file mode 100644
index 9de2551..0000000
--- a/slider-core/src/main/java/org/apache/slider/client/SliderRestClient.java
+++ /dev/null
@@ -1,343 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.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.commons.lang.StringUtils;
-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;
-    if (appmaster !=null) {
-      bindToAppmaster(appmaster);
-    }
-  }
-  
-  public SliderRestClient(Client jersey, WebResource appmaster) {
-    this("SliderRestClient", jersey, appmaster);
-  }
-
-  public Client getJersey() {
-    return jersey;
-  }
-
-  /**
-   * Bind/rebind to the AM
-   * @param appmaster AM
-   */
-  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) {
-    Preconditions.checkArgument(!StringUtils.isEmpty(subpath),
-        "empty path");
-    Preconditions.checkNotNull(appResource, "Null app resource");
-    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_CONTAINERS,
-        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/b0963bd0/slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java b/slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java
new file mode 100644
index 0000000..46be2aa
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java
@@ -0,0 +1,154 @@
+/*
+ * 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.rest;
+
+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 org.apache.slider.core.restclient.HttpVerb;
+import org.apache.slider.core.restclient.UgiJerseyBinding;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.net.URI;
+
+
+/**
+ * This is a base class for Jersey Rest clients in Slider.
+ * It supports bonding to an AM and the execution of operations —with
+ * exceptions uprated to IOExceptions when needed.
+ * <p>
+ * Subclasses can use these operations to provide an API-like view
+ * of the REST model
+ */
+public class BaseRestClient  {
+  private static final Logger log =
+      LoggerFactory.getLogger(BaseRestClient.class);
+  private final Client client;
+  private WebResource appmaster;
+
+  public BaseRestClient(
+      Client client,
+      WebResource appmaster) {
+    Preconditions.checkNotNull(client, "null jersey client");
+    this.client = client;
+    if (appmaster != null) {
+      bindToAppmaster(appmaster);
+    }
+  }
+  
+  public Client getClient() {
+    return client;
+  }
+
+  /**
+   * Bind/rebind to the AM
+   * @param appmaster AM
+   */
+  public void bindToAppmaster(WebResource appmaster) {
+    Preconditions.checkArgument(appmaster != null, " Null appmaster");
+    this.appmaster = appmaster;
+  }
+
+  public WebResource getAppmaster() {
+    return appmaster;
+  }
+
+  /**
+   * 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 t 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);
+    }
+  }
+
+  /**
+   * Create a Web resource from the client.
+   *
+   * @param u the URI of the resource.
+   * @return the Web resource.
+   */
+  protected WebResource resource(URI u) {
+    return client.resource(u);
+  }
+
+  /**
+   * Create a Web resource from the client.
+   *
+   * @param u the URI of the resource.
+   * @return the Web resource.
+   */
+
+  protected WebResource resource(String url) {
+    WebResource resource = client.resource(url);
+    return resource;
+  }
+
+  protected WebResource jsonResource(String url) {
+    WebResource resource = resource(url);
+    resource.type(MediaType.APPLICATION_JSON);
+    return resource;
+  }
+
+
+  protected WebResource jsonResource(URI u) {
+    WebResource resource = resource(u);
+    resource.type(MediaType.APPLICATION_JSON);
+    return resource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b0963bd0/slider-core/src/main/java/org/apache/slider/client/rest/SliderApplicationAPI.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/rest/SliderApplicationAPI.java b/slider-core/src/main/java/org/apache/slider/client/rest/SliderApplicationAPI.java
new file mode 100644
index 0000000..16af359
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/client/rest/SliderApplicationAPI.java
@@ -0,0 +1,274 @@
+/*
+ * 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.rest;
+
+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.WebResource;
+import com.sun.jersey.api.representation.Form;
+import org.apache.commons.lang.StringUtils;
+import org.apache.slider.api.types.SerializedComponentInformation;
+import org.apache.slider.api.types.SerializedContainerInformation;
+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.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 SliderApplicationAPI extends BaseRestClient {
+  private static final Logger log =
+      LoggerFactory.getLogger(SliderApplicationAPI.class);
+  private WebResource appResource;
+  
+  public SliderApplicationAPI(Client jerseyClient, WebResource appmaster) {
+    super(jerseyClient, appmaster);
+  }
+
+  /**
+   * Bind/rebind to the AM
+   * @param appmaster AM
+   */
+  public void bindToAppmaster(WebResource appmaster) {
+    super.bindToAppmaster(appmaster);
+    this.appResource = appmaster.path(SLIDER_PATH_APPLICATION);
+  }
+
+  /**
+   * Create a resource under the application path
+   * @param subpath
+   * @return an resource under the application path
+   */
+  public WebResource applicationResource(String subpath) {
+    Preconditions.checkArgument(!StringUtils.isEmpty(subpath),
+        "empty path");
+    Preconditions.checkNotNull(appResource, "Null app resource");
+    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);
+  }
+
+
+  /**
+   * 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_CONTAINERS,
+        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/b0963bd0/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
index 46dba1d..71f3933 100644
--- 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
@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.PathNotFoundException
 import org.apache.slider.api.StateValues
 import org.apache.slider.api.types.SerializedComponentInformation
 import org.apache.slider.api.types.SerializedContainerInformation
-import org.apache.slider.client.SliderRestClient
+import org.apache.slider.client.rest.SliderApplicationAPI
 import org.apache.slider.core.conf.ConfTree
 import org.apache.slider.core.conf.ConfTreeOperations
 import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource
@@ -55,7 +55,7 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
   final String appmaster;
   final String application;
   final Client jersey;
-  final SliderRestClient restClient;
+  final SliderApplicationAPI appAPI;
 
 
   SliderRestClientTestDelegates(String appmaster, Client jersey) {
@@ -64,27 +64,27 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     application = appendToURL(appmaster, SLIDER_PATH_APPLICATION)
     WebResource amResource = jersey.resource(appmaster)
     amResource.type(MediaType.APPLICATION_JSON)
-    restClient = new SliderRestClient(jersey, amResource)
+    appAPI = new SliderApplicationAPI(jersey, amResource)
   }
 
 
   public void testGetDesiredModel() throws Throwable {
-      restClient.getDesiredModel()  
-      restClient.getDesiredAppconf()  
-      restClient.getDesiredYarnResources()  
+      appAPI.getDesiredModel()  
+      appAPI.getDesiredAppconf()  
+      appAPI.getDesiredYarnResources()  
   }
 
   public void testGetResolvedModel() throws Throwable {
-      restClient.getResolvedModel()  
-      restClient.getResolvedAppconf()  
-      restClient.getResolvedYarnResources()  
+      appAPI.getResolvedModel()  
+      appAPI.getResolvedAppconf()  
+      appAPI.getResolvedYarnResources()  
   }
 
   
   public void testLiveResources() throws Throwable {
     describe "Live Resources"
 
-    ConfTreeOperations tree = restClient.getLiveYarnResources()
+    ConfTreeOperations tree = appAPI.getLiveYarnResources()
 
     log.info tree.toString()
     def liveAM = tree.getComponent(COMPONENT_AM)
@@ -102,7 +102,7 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
   public void testLiveContainers() throws Throwable {
     describe "Application REST ${LIVE_CONTAINERS}"
 
-    Map<String, SerializedContainerInformation> containers = restClient.enumContainers()
+    Map<String, SerializedContainerInformation> containers = appAPI.enumContainers()
     assert containers.size() == 1
     log.info "${containers}"
     SerializedContainerInformation amContainerInfo =
@@ -122,12 +122,12 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     describe "containers"
 
     SerializedContainerInformation amContainerInfo2 =
-        restClient.getContainer(amContainerId)
+        appAPI.getContainer(amContainerId)
     assert amContainerInfo2.containerId == amContainerId
 
     // fetch missing
     try {
-      def result = restClient.getContainer("/unknown")
+      def result = appAPI.getContainer("/unknown")
       fail("expected an error, got $result")
     } catch (PathNotFoundException e) {
       // expected
@@ -137,7 +137,7 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     describe "components"
 
     Map<String, SerializedComponentInformation> components =
-        restClient.enumComponents()
+        appAPI.enumComponents()
 
     // two components
     assert components.size() >= 1
@@ -146,7 +146,7 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     SerializedComponentInformation amComponentInfo =
         (SerializedComponentInformation) components[COMPONENT_AM]
 
-    SerializedComponentInformation amFullInfo = restClient.getComponent(COMPONENT_AM) 
+    SerializedComponentInformation amFullInfo = appAPI.getComponent(COMPONENT_AM) 
 
     assert amFullInfo.containers.size() == 1
     assert amFullInfo.containers[0] == amContainerId
@@ -166,14 +166,14 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
         MODEL,
         ApplicationResource.MODEL_ENTRIES)
 
-    def unresolvedConf = restClient.getDesiredModel() 
+    def unresolvedConf = appAPI.getDesiredModel() 
 //    log.info "Unresolved \n$unresolvedConf"
     def unresolvedAppConf = unresolvedConf.appConfOperations
 
     def sam = "slider-appmaster"
     assert unresolvedAppConf.getComponentOpt(sam,
         TEST_GLOBAL_OPTION, "") == ""
-    def resolvedConf = restClient.getResolvedModel() 
+    def resolvedConf = appAPI.getResolvedModel() 
     assert resolvedConf.appConfOperations.getComponentOpt(
         sam, TEST_GLOBAL_OPTION, "") == TEST_GLOBAL_OPTION_PRESENT
 
@@ -184,7 +184,7 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
 
 
     
-    def resolvedAppconf = restClient.getResolvedAppconf() 
+    def resolvedAppconf = appAPI.getResolvedAppconf() 
     assert resolvedAppconf.
                components[sam][TEST_GLOBAL_OPTION] == TEST_GLOBAL_OPTION_PRESENT
   }
@@ -193,7 +193,7 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
     // GET
     describe "pinging"
     
-    restClient.ping("hello")
+    appAPI.ping("hello")
   }
 
 


[7/7] incubator-slider git commit: Merge branch 'feature/SLIDER-762_Add_API-level_slider_REST_client' into develop

Posted by st...@apache.org.
Merge branch 'feature/SLIDER-762_Add_API-level_slider_REST_client' into develop


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

Branch: refs/heads/develop
Commit: 6bd71659e932fb7e6a29777599a650df26214096
Parents: ea6eca1 beda1b2
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 23 19:43:08 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 23 19:43:08 2015 +0000

----------------------------------------------------------------------
 .../slider/client/rest/BaseRestClient.java      | 154 +++++++++++
 .../slider/client/rest/RestClientFactory.java   |  61 +++++
 .../client/rest/RestClientRegistryBinder.java   | 201 ++++++++++++++
 .../client/rest/SliderApplicationAPI.java       | 274 +++++++++++++++++++
 .../core/exceptions/ExceptionConverter.java     |  20 +-
 .../registry/info/CustomRegistryConstants.java  |   5 +-
 .../registry/retrieve/RegistryRetriever.java    |  67 +----
 .../core/restclient/UgiJerseyBinding.java       |  32 ++-
 .../restclient/UrlConnectionOperations.java     |  94 +++++--
 .../slideram/SliderAMProviderService.java       |   5 +
 .../server/appmaster/state/RoleStatus.java      |   5 +
 .../rest/application/ApplicationResource.java   |   2 +-
 .../agent/rest/JerseyTestDelegates.groovy       |  24 +-
 .../rest/SliderRestClientTestDelegates.groovy   | 251 +++++++++++++++++
 .../slider/agent/rest/TestStandaloneREST.groovy |  37 ++-
 .../funtest/lifecycle/AgentWebPagesIT.groovy    |  46 +++-
 16 files changed, 1183 insertions(+), 95 deletions(-)
----------------------------------------------------------------------



[6/7] incubator-slider git commit: SLIDER-762 Rest client has ability to bind via YARN registry

Posted by st...@apache.org.
SLIDER-762 Rest client has ability to bind via YARN registry


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

Branch: refs/heads/develop
Commit: beda1b2d83c214fced12eae801ad966dfc1a173b
Parents: 6c038b4
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 23 19:42:08 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 23 19:42:08 2015 +0000

----------------------------------------------------------------------
 .../slider/client/rest/RestClientFactory.java   |  61 ++++++
 .../client/rest/RestClientRegistryBinder.java   | 201 +++++++++++++++++++
 .../core/exceptions/ExceptionConverter.java     |  20 +-
 .../registry/info/CustomRegistryConstants.java  |   5 +-
 .../registry/retrieve/RegistryRetriever.java    |  67 ++-----
 .../core/restclient/UgiJerseyBinding.java       |  35 +---
 .../slideram/SliderAMProviderService.java       |   5 +
 .../rest/SliderRestClientTestDelegates.groovy   |   4 +-
 .../slider/agent/rest/TestStandaloneREST.groovy |  29 ++-
 .../funtest/lifecycle/AgentWebPagesIT.groovy    |  32 ++-
 10 files changed, 358 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/slider-core/src/main/java/org/apache/slider/client/rest/RestClientFactory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/rest/RestClientFactory.java b/slider-core/src/main/java/org/apache/slider/client/rest/RestClientFactory.java
new file mode 100644
index 0000000..1b7553e
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/client/rest/RestClientFactory.java
@@ -0,0 +1,61 @@
+/*
+ * 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.rest;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.WebResource;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.slider.core.registry.info.CustomRegistryConstants;
+
+import java.io.IOException;
+
+public class RestClientFactory {
+
+  private final RestClientRegistryBinder binder;
+  private final Client jerseyClient;
+  private final String user, serviceclass, instance;
+
+  public RestClientFactory(RegistryOperations operations,
+      Client jerseyClient,
+      String user,
+      String serviceclass,
+      String instance) {
+    this.jerseyClient = jerseyClient;
+    this.user = user;
+    this.serviceclass = serviceclass;
+    this.instance = instance;
+    binder = new RestClientRegistryBinder(operations);
+  }
+
+  /**
+   * Locate the AM
+   * @return a resource to the AM
+   * @throws IOException any failure to resolve to the AM
+   */
+  WebResource locateAppmaster() throws IOException {
+    String restAPI = binder.lookupExternalRestAPI(user, serviceclass, instance,
+        CustomRegistryConstants.AM_REST_BASE);
+    return jerseyClient.resource(restAPI);
+  }
+  
+  public SliderApplicationAPI createSliderApplicationApi() throws IOException {
+    return new SliderApplicationAPI(jerseyClient, locateAppmaster());
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/slider-core/src/main/java/org/apache/slider/client/rest/RestClientRegistryBinder.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/rest/RestClientRegistryBinder.java b/slider-core/src/main/java/org/apache/slider/client/rest/RestClientRegistryBinder.java
new file mode 100644
index 0000000..048a07b
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/client/rest/RestClientRegistryBinder.java
@@ -0,0 +1,201 @@
+/*
+ * 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.rest;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.impl.zk.RegistryInternalConstants;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.registry.client.binding.RegistryPathUtils.encodeForRegistry;
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.convertUsername;
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.getCurrentUsernameUnencoded;
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.servicePath;
+
+/**
+ * Generic code to get the URLs for clients via the registry
+ */
+public class RestClientRegistryBinder {
+  private static final Logger log =
+      LoggerFactory.getLogger(RestClientRegistryBinder.class);
+
+  private final RegistryOperations operations;
+
+  public RestClientRegistryBinder(RegistryOperations operations) {
+    this.operations = operations;
+  }
+
+  /**
+   * Buld the user path -switches to the system path if the user is "".
+   * It also cross-converts the username to ascii via punycode
+   * @param username username or ""
+   * @return the path to the user
+   */
+  public static String homePathForUser(String username) {
+    Preconditions.checkArgument(username != null, "null user");
+
+    // catch recursion
+    if (username.startsWith(RegistryConstants.PATH_USERS)) {
+      return username;
+    }
+
+    if (username.isEmpty()) {
+      return RegistryConstants.PATH_SYSTEM_SERVICES;
+    }
+
+    // convert username to registry name
+    String convertedName = convertUsername(username);
+
+    return RegistryPathUtils.join(RegistryConstants.PATH_USERS,
+        encodeForRegistry(convertedName));
+  }
+
+  /**
+   * Get the current username, before any encoding has been applied.
+   * @return the current user from the kerberos identity, falling back
+   * to the user and/or env variables.
+   */
+  public static String currentUsernameUnencoded() {
+    String env_hadoop_username = System.getenv(
+        RegistryInternalConstants.HADOOP_USER_NAME);
+    return getCurrentUsernameUnencoded(env_hadoop_username);
+  }
+
+  /**
+   * Qualify a user.
+   * <ol>
+   *   <li> <code>"~"</code> maps to user home path home</li>
+   *   <li> <code>"~user"</code> maps to <code>/users/$user</code></li>
+   *   <li> <code>"/"</code> maps to <code>/services/</code></li>
+   * </ol>
+   * @param user the username
+   * @return the base path
+   */
+  public static String qualifyUser(String user) {
+    // qualify the user
+    String t = user.trim();
+    if (t.startsWith("/")) {
+      // already resolved
+      return t;
+    } else if (t.equals("~")) {
+      // self
+      return currentUsernameUnencoded();
+    } else if (t.startsWith("~")) {
+      // another user
+      // convert username to registry name
+      String convertedName = convertUsername(t.substring(1));
+
+      return RegistryPathUtils.join(RegistryConstants.PATH_USERS,
+          encodeForRegistry(convertedName));
+    } else {
+      return "/" + t;
+    }
+  }
+
+  /**
+   * Look up an external REST API
+   * @param user user which will be qualified as per {@link #qualifyUser(String)}
+   * @param serviceClass service class
+   * @param instance instance name
+   * @param api API
+   * @return the API, or an exception is raised.
+   * @throws IOException
+   */
+  public String lookupExternalRestAPI(String user,
+      String serviceClass,
+      String instance,
+      String api)
+      throws IOException {
+    String qualified = qualifyUser(user);
+    String path = servicePath(qualified, serviceClass, instance);
+    String restAPI = resolveExternalRestAPI(api, path);
+    if (restAPI == null) {
+      throw new PathNotFoundException(path + " API " + api);
+    }
+    return restAPI;
+  }
+
+  /**
+   * Resolve a service record then return an external REST API exported it.
+   *
+   * @param api API to resolve
+   * @param path path of the service record
+   * @return null if the record exists but the API is absent or it has no
+   * REST endpoints.
+   * @throws IOException resolution problems, as covered in
+   * {@link RegistryOperations#resolve(String)}
+   */
+  protected String resolveExternalRestAPI(String api, String path) throws
+      IOException {
+    ServiceRecord record = operations.resolve(path);
+    return lookupRestAPI(record, api, true);
+  }
+
+  /**
+   * Look up an external REST API endpoint
+   * @param record service record
+   * @param api URI of api
+   * @param external flag to indicate this is an external record
+   * @return the first endpoint of the implementation, or null if there
+   * is no entry for the API, implementation or it's the wrong type.
+   */
+  public static String lookupRestAPI(ServiceRecord record,
+      String api, boolean external) throws InvalidRecordException {
+    try {
+      String url = null;
+      Endpoint endpoint = getEndpoint(record, api, external);
+      List<String> addresses =
+          RegistryTypeUtils.retrieveAddressesUriType(endpoint);
+      if (addresses != null && !addresses.isEmpty()) {
+        url = addresses.get(0);
+      }
+      return url;
+    } catch (InvalidRecordException e) {
+      log.debug("looking for API {}", api, e);
+      return null;
+    }
+  }
+
+  /**
+   * Get an endpont by API
+   * @param record service record
+   * @param api API
+   * @param external flag to indicate this is an external record
+   * @return the endpoint or null
+   */
+  public static Endpoint getEndpoint(ServiceRecord record,
+      String api,
+      boolean external) {
+    return external ? record.getExternalEndpoint(api)
+                    : record.getInternalEndpoint(api);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java b/slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
index ed5ceb2..e3cd508 100644
--- a/slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
+++ b/slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
@@ -21,7 +21,10 @@ package org.apache.slider.core.exceptions;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import org.apache.hadoop.fs.PathAccessDeniedException;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.yarn.webapp.*;
 
+import javax.servlet.http.HttpServletResponse;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
@@ -32,19 +35,27 @@ import java.io.IOException;
 public class ExceptionConverter {
 
   /**
-   * Convert a Jersey Exception into an IOE or subclass
+   * Uprate error codes 400 and up into faults; 
+   * 404 is converted to a {@link FileNotFoundException},
+   * 401 to {@link ForbiddenException}
+   * FileNotFoundException for an unknown resource
+   * PathAccessDeniedException for access denied
+   * PathIOException for anything else
+   * @param verb HTTP Verb used
    * @param targetURL URL being targeted 
    * @param exception original exception
    * @return a new exception, the original one nested as a cause
    */
-  public static IOException convertJerseyException(String targetURL,
+  public static IOException convertJerseyException(String verb,
+      String targetURL,
       UniformInterfaceException exception) {
 
     IOException ioe = null;
     ClientResponse response = exception.getResponse();
     if (response != null) {
       int status = response.getStatus();
-      if (status == 401) {
+      if (status == HttpServletResponse.SC_UNAUTHORIZED
+          || status == HttpServletResponse.SC_FORBIDDEN) {
         ioe = new PathAccessDeniedException(targetURL);
       }
       if (status >= 400 && status < 500) {
@@ -53,7 +64,8 @@ public class ExceptionConverter {
     }
 
     if (ioe == null) {
-      ioe = new IOException("Failed to GET " + targetURL + ": " + exception);
+      ioe = new PathIOException(targetURL, 
+          verb + " " + targetURL + " failed: " + exception);
     }
     ioe.initCause(exception);
     return ioe; 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java b/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
index 0a3476c..13ad5c5 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
@@ -48,7 +48,10 @@ public class CustomRegistryConstants {
       "classpath:org.apache.slider.agents.oneway";
 
   public static final String AM_IPC_PROTOCOL =
-      "classpath:org.apache.slider.appmaster";
+      "classpath:org.apache.slider.appmaster.ipc";
+
+  public static final String AM_REST_BASE =
+      "classpath:org.apache.slider.client.rest";
 
   public static final String WEB_UI = "http://";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java b/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
index 0d32807..3ed7cc6 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
@@ -21,23 +21,21 @@ package org.apache.slider.core.registry.retrieve;
 import com.beust.jcommander.Strings;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
-import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
 import org.apache.hadoop.registry.client.exceptions.RegistryIOException;
-import org.apache.hadoop.registry.client.types.Endpoint;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
+import static org.apache.slider.client.rest.RestClientRegistryBinder.*;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.exceptions.ExceptionConverter;
 import org.apache.slider.core.registry.docstore.PublishedConfigSet;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
 import org.apache.slider.core.registry.docstore.PublishedExports;
 import org.apache.slider.core.registry.docstore.PublishedExportsSet;
-import org.apache.slider.core.registry.info.CustomRegistryConstants;
+import static org.apache.slider.core.registry.info.CustomRegistryConstants.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.List;
 
 /**
  * Registry retriever. 
@@ -69,51 +67,14 @@ public class RegistryRetriever extends AMWebClient {
    * not match that expected (i.e. not a list of URLs), missing endpoint...
    */
   public RegistryRetriever(ServiceRecord record) throws RegistryIOException {
-    Endpoint internal = record.getInternalEndpoint(
-        CustomRegistryConstants.PUBLISHER_CONFIGURATIONS_API);
-    String url = null;
-    if (internal != null) {
-      List<String> addresses = RegistryTypeUtils.retrieveAddressesUriType(
-          internal);
-      if (addresses != null && !addresses.isEmpty()) {
-        url = addresses.get(0);
-      }
-    }
-    internalConfigurationURL = url;
-    Endpoint external = record.getExternalEndpoint(
-        CustomRegistryConstants.PUBLISHER_CONFIGURATIONS_API);
-    url = null;
-    if (external != null) {
-      List<String> addresses =
-          RegistryTypeUtils.retrieveAddressesUriType(external);
-      if (addresses != null && !addresses.isEmpty()) {
-        url = addresses.get(0);
-      }
-    }
-    externalConfigurationURL = url;
-
-    internal = record.getInternalEndpoint(
-        CustomRegistryConstants.PUBLISHER_EXPORTS_API);
-    url = null;
-    if (internal != null) {
-      List<String> addresses = RegistryTypeUtils.retrieveAddressesUriType(
-          internal);
-      if (addresses != null && !addresses.isEmpty()) {
-        url = addresses.get(0);
-      }
-    }
-    internalExportsURL = url;
-    external = record.getExternalEndpoint(
-        CustomRegistryConstants.PUBLISHER_EXPORTS_API);
-    url = null;
-    if (external != null) {
-      List<String> addresses =
-          RegistryTypeUtils.retrieveAddressesUriType(external);
-      if (addresses != null && !addresses.isEmpty()) {
-        url = addresses.get(0);
-      }
-    }
-    externalExportsURL = url;
+    internalConfigurationURL = lookupRestAPI(record,
+        PUBLISHER_CONFIGURATIONS_API, true);
+    externalConfigurationURL = lookupRestAPI(record,
+        PUBLISHER_CONFIGURATIONS_API, false);
+    externalExportsURL = lookupRestAPI(record,
+        PUBLISHER_EXPORTS_API, true);
+    internalExportsURL = lookupRestAPI(record,
+        PUBLISHER_EXPORTS_API, false);
   }
 
   /**
@@ -141,7 +102,7 @@ public class RegistryRetriever extends AMWebClient {
       PublishedConfigSet configSet = webResource.get(PublishedConfigSet.class);
       return configSet;
     } catch (UniformInterfaceException e) {
-      throw ExceptionConverter.convertJerseyException(confURL, e);
+      throw ExceptionConverter.convertJerseyException("GET", confURL, e);
     }
   }
 
@@ -176,7 +137,7 @@ public class RegistryRetriever extends AMWebClient {
       PublishedExportsSet exportSet = webResource.get(PublishedExportsSet.class);
       return exportSet;
     } catch (UniformInterfaceException e) {
-      throw ExceptionConverter.convertJerseyException(exportsUrl, e);
+      throw ExceptionConverter.convertJerseyException("GET", exportsUrl, e);
     }
   }
 
@@ -203,7 +164,7 @@ public class RegistryRetriever extends AMWebClient {
           webResource.get(PublishedConfiguration.class);
       return publishedConf;
     } catch (UniformInterfaceException e) {
-      throw ExceptionConverter.convertJerseyException(confURL, e);
+      throw ExceptionConverter.convertJerseyException("GET", confURL, e);
     }
   }
 
@@ -229,7 +190,7 @@ public class RegistryRetriever extends AMWebClient {
           webResource.get(PublishedExports.class);
       return publishedExports;
     } catch (UniformInterfaceException e) {
-      throw ExceptionConverter.convertJerseyException(exportsURL, e);
+      throw ExceptionConverter.convertJerseyException("GET", exportsURL, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/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 10ce1ce..cad3c69 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
@@ -30,10 +30,12 @@ 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.apache.slider.core.exceptions.ExceptionConverter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.servlet.http.HttpServletResponse;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.URL;
@@ -112,39 +114,14 @@ public class UgiJerseyBinding implements
 
   /**
    * 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
-   * @throws PathNotFoundException for an unknown resource
-   * @throws PathAccessDeniedException for access denied
-   * @throws PathIOException for anything else
+   * <p>
+   * see {@link ExceptionConverter#convertJerseyException(String, String, UniformInterfaceException)}
    */
   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 == HttpServletResponse.SC_NOT_FOUND) {
-      return (IOException) new PathNotFoundException(url).initCause(ex);
-    }
-    if (resultCode == HttpServletResponse.SC_UNAUTHORIZED
-        || resultCode == HttpServletResponse.SC_FORBIDDEN) {
-      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);
+    return ExceptionConverter.convertJerseyException(verb.getVerb(),
+        url, ex);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
index afe6428..408e311 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
@@ -152,6 +152,11 @@ public class SliderAMProviderService extends AbstractProviderService implements
       serviceRecord.addExternalEndpoint(
           RegistryTypeUtils.webEndpoint(
               CustomRegistryConstants.WEB_UI, amWebURI.toURI()));
+      
+      serviceRecord.addExternalEndpoint(
+          RegistryTypeUtils.webEndpoint(
+              CustomRegistryConstants.AM_REST_BASE, amWebURI.toURI()));
+      
       serviceRecord.addExternalEndpoint(
           RegistryTypeUtils.restEndpoint(
               CustomRegistryConstants.MANAGEMENT_REST_API,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/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
index 71f3933..ef070e0 100644
--- 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
@@ -127,9 +127,9 @@ class SliderRestClientTestDelegates extends SliderTestUtils {
 
     // fetch missing
     try {
-      def result = appAPI.getContainer("/unknown")
+      def result = appAPI.getContainer("unknown")
       fail("expected an error, got $result")
-    } catch (PathNotFoundException e) {
+    } catch (FileNotFoundException e) {
       // expected
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/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 8ee88c3..3c3dd52 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
@@ -28,12 +28,17 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.commons.httpclient.HttpClient
 import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager
+import org.apache.hadoop.registry.client.api.RegistryOperations
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 import org.apache.slider.agent.AgentMiniClusterTestBase
 import org.apache.slider.client.SliderClient
+import org.apache.slider.client.rest.RestClientFactory
+import org.apache.slider.client.rest.SliderApplicationAPI
+import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.core.main.ServiceLauncher
+import org.apache.slider.core.registry.info.CustomRegistryConstants
 import org.apache.slider.core.restclient.HttpOperationResponse
 import org.junit.Test
 
@@ -109,16 +114,16 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     log.info "Content type: ${response.contentType}"
 
     
-    
+    def ugiClient = createUGIJerseyClient();
     
     describe "Proxy SliderRestClient Tests"
     SliderRestClientTestDelegates proxySliderRestClient =
-        new SliderRestClientTestDelegates(proxyAM, createUGIJerseyClient())
+        new SliderRestClientTestDelegates(proxyAM, ugiClient)
     proxySliderRestClient.testSuiteGetOperations()
 
     describe "Direct SliderRestClient Tests"
     SliderRestClientTestDelegates directSliderRestClient =
-        new SliderRestClientTestDelegates(directAM, createUGIJerseyClient())
+        new SliderRestClientTestDelegates(directAM, ugiClient)
     directSliderRestClient.testSuiteGetOperations()
     directSliderRestClient.testSuiteComplexVerbs()
 
@@ -126,13 +131,13 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     
     describe "Proxy Jersey Tests"
     JerseyTestDelegates proxyJerseyTests =
-        new JerseyTestDelegates(proxyAM, createUGIJerseyClient())
+        new JerseyTestDelegates(proxyAM, ugiClient)
     proxyJerseyTests.testSuiteGetOperations()
 
     describe "Direct Jersey Tests"
 
     JerseyTestDelegates directJerseyTests =
-        new JerseyTestDelegates(directAM, createUGIJerseyClient())
+        new JerseyTestDelegates(directAM, ugiClient)
     directJerseyTests.testSuiteGetOperations()
     directJerseyTests.testSuiteComplexVerbs()
 
@@ -150,7 +155,19 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
       // and via the proxy
       proxied.testSuiteComplexVerbs()
     }
-    
+
+    // create the Rest client via the registry
+
+    RegistryOperations operations = client.registryOperations;
+    def restClientFactory = new RestClientFactory(
+        operations,
+        ugiClient,
+        "~", SliderKeys.APP_TYPE,
+        clustername)
+    def sliderApplicationApi = restClientFactory.createSliderApplicationApi();
+    sliderApplicationApi.desiredModel
+    sliderApplicationApi.resolvedModel
+    sliderApplicationApi.ping("registry located")
 
 /*    DISABLED: this client does not pass the tests.
     

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/beda1b2d/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 fd54b55..e6b8bc4 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
@@ -18,14 +18,19 @@
 
 package org.apache.slider.funtest.lifecycle
 
+import com.sun.jersey.api.client.Client
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.registry.client.api.RegistryOperations
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.webapp.ForbiddenException
 import org.apache.slider.agent.rest.JerseyTestDelegates
 import org.apache.slider.agent.rest.RestTestDelegates
 import org.apache.slider.agent.rest.SliderRestClientTestDelegates
+import org.apache.slider.client.SliderClient
+import org.apache.slider.client.rest.RestClientFactory
 import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -118,26 +123,27 @@ public class AgentWebPagesIT extends AgentCommandTestBase
 
     describe "Proxy Jersey Tests"
 
+    Client ugiClient = createUGIJerseyClient()
     JerseyTestDelegates proxyJerseyTests =
-        new JerseyTestDelegates(proxyAM, createUGIJerseyClient())
+        new JerseyTestDelegates(proxyAM, ugiClient)
     proxyJerseyTests.testSuiteGetOperations()
 
     describe "Direct Jersey Tests"
     JerseyTestDelegates directJerseyTests =
-        new JerseyTestDelegates(directAM, createUGIJerseyClient())
+        new JerseyTestDelegates(directAM, ugiClient)
     directJerseyTests.testSuiteGetOperations()
     directJerseyTests.testSuiteComplexVerbs()
 
     describe "Proxy SliderRestClient Tests"
     SliderRestClientTestDelegates proxySliderRestClient =
-        new SliderRestClientTestDelegates(proxyAM, createUGIJerseyClient())
+        new SliderRestClientTestDelegates(proxyAM, ugiClient)
     proxySliderRestClient.testSuiteGetOperations()
     if (!wsBackDoorRequired) {
       proxySliderRestClient.testSuiteComplexVerbs()
     }
     describe "Direct SliderRestClient Tests"
     SliderRestClientTestDelegates directSliderRestClient =
-        new SliderRestClientTestDelegates(directAM, createUGIJerseyClient())
+        new SliderRestClientTestDelegates(directAM, ugiClient)
     directSliderRestClient.testSuiteGetOperations()
     directSliderRestClient.testSuiteComplexVerbs()
 
@@ -155,10 +161,24 @@ public class AgentWebPagesIT extends AgentCommandTestBase
       
       // these tests use the Jersey client without the Hadoop-specific
       // SPNEGO
-      JerseyTestDelegates baseicJerseyClientTests =
+      JerseyTestDelegates basicJerseyClientTests =
           new JerseyTestDelegates(proxyAM, createBasicJerseyClient())
-      baseicJerseyClientTests.testSuiteGetOperations()
+      basicJerseyClientTests.testSuiteGetOperations()
     }
+
+    // create the Rest client via the registry
+
+    //get a slider client against the cluster
+
+    SliderClient sliderClient = bondToCluster(SLIDER_CONFIG, CLUSTER)
+    RegistryOperations operations = sliderClient.registryOperations;
+    def restClientFactory = new RestClientFactory(
+        operations, ugiClient,
+        "~", SliderKeys.APP_TYPE, CLUSTER)
+    def sliderApplicationApi = restClientFactory.createSliderApplicationApi();
+    sliderApplicationApi.desiredModel
+    sliderApplicationApi.resolvedModel
+    sliderApplicationApi.ping("registry located")
     
     // finally, stop the AM
     direct.testStop();