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

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

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