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:20 UTC

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

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")
   }