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

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

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