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

[10/28] incubator-slider git commit: SLIDER-719 REST client tested on secure clusters. The jersey + httpclient combination does not work, but the others do

SLIDER-719 REST client tested on secure clusters. The jersey + httpclient combination does not work, but the others do


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

Branch: refs/heads/develop
Commit: 0097ce923a217db00095fba6273ce7f339064cb7
Parents: 4418179
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 21 17:54:39 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 21 17:54:39 2015 +0000

----------------------------------------------------------------------
 pom.xml                                         |  6 ++
 slider-core/pom.xml                             |  6 ++
 .../core/restclient/HttpOperationResponse.java  |  5 ++
 .../restclient/UrlConnectionOperations.java     |  2 +
 .../rest/application/ApplicationResource.java   |  1 -
 .../agent/rest/JerseyTestDelegates.groovy       | 25 ++++--
 .../slider/agent/rest/RestTestDelegates.groovy  | 20 ++++-
 .../slider/agent/rest/TestStandaloneREST.groovy | 91 +++++++++++++++++---
 .../apache/slider/test/SliderTestUtils.groovy   | 44 ++++++++--
 .../funtest/lifecycle/AgentWebPagesIT.groovy    | 62 +++++++++----
 10 files changed, 212 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b206d66..5c18f77 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1204,6 +1204,12 @@
       </dependency>
 
       <dependency>
+        <groupId>com.sun.jersey.contribs</groupId>
+        <artifactId>jersey-apache-client</artifactId>
+        <version>${jersey.version}</version>
+      </dependency>
+
+      <dependency>
         <groupId>com.sun.jersey</groupId>
         <artifactId>jersey-json</artifactId>
         <version>${jersey.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/slider-core/pom.xml b/slider-core/pom.xml
index be4d95c..43a1193 100644
--- a/slider-core/pom.xml
+++ b/slider-core/pom.xml
@@ -407,6 +407,12 @@
     </dependency>
 
     <dependency>
+      <groupId>com.sun.jersey.contribs</groupId>
+      <artifactId>jersey-apache-client</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.inject</groupId>
       <artifactId>guice</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java
index a5357a2..0266223 100644
--- a/slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java
@@ -18,12 +18,17 @@
 
 package org.apache.slider.core.restclient;
 
+import java.util.List;
+import java.util.Map;
+
 /**
  * A response for use as a return value from operations
  */
 public class HttpOperationResponse {
   
   public int responseCode;
+  public long lastModified;
   public String contentType;
   public byte[] data;
+  public Map<String, List<String>> headers;
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/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 65a4117..6e29c55 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
@@ -133,7 +133,9 @@ public class UrlConnectionOperations extends Configured  {
       }
       
       resultCode = conn.getResponseCode();
+      outcome.lastModified = conn.getLastModified();
       outcome.contentType = conn.getContentType();
+      outcome.headers = conn.getHeaderFields();
       InputStream stream = conn.getErrorStream();
       if (stream == null) {
         stream = conn.getInputStream();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/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 d72a486..c2c83a0 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
@@ -366,7 +366,6 @@ public class ApplicationResource extends AbstractSliderResource {
   
   @HEAD
   @Path(ACTION_PING)
-  @Produces({APPLICATION_JSON})
   public Object actionPingHead(@Context HttpServletRequest request,
       @Context UriInfo uriInfo) {
     mark("HEAD", SLIDER_SUBPATH_APPLICATION, ACTION_PING);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/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 0699800..09788e1 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
@@ -19,8 +19,10 @@
 package org.apache.slider.agent.rest
 
 import com.sun.jersey.api.client.Client
-import com.sun.jersey.api.client.WebResource
+import com.sun.jersey.api.client.ClientResponse
 import com.sun.jersey.api.client.UniformInterfaceException
+import com.sun.jersey.api.client.WebResource
+import com.sun.jersey.client.impl.ClientRequestImpl
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.webapp.NotFoundException
@@ -30,14 +32,10 @@ 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.HttpOperationResponse
 import org.apache.slider.core.restclient.HttpVerb
-import org.apache.slider.server.appmaster.web.rest.RestPaths
 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.Outcome
 import org.apache.slider.test.SliderTestUtils
-import org.glassfish.grizzly.servlet.ver25.WebResourceCollectionType
 
 import javax.ws.rs.core.MediaType
 
@@ -99,6 +97,7 @@ class JerseyTestDelegates extends SliderTestUtils {
   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)
     } catch (UniformInterfaceException ex) {
       uprateFaults(method, resource, ex)
@@ -203,8 +202,21 @@ class JerseyTestDelegates extends SliderTestUtils {
     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()
@@ -267,7 +279,7 @@ class JerseyTestDelegates extends SliderTestUtils {
     Map<String, SerializedComponentInformation> components =
         jFetchType(LIVE_COMPONENTS, HashMap)
     // two components
-    assert components.size() == 1
+    assert components.size() >= 1
     log.info "${components}"
 
     SerializedComponentInformation amComponentInfo =
@@ -451,6 +463,7 @@ class JerseyTestDelegates extends SliderTestUtils {
   public void testSuiteGetOperations() {
 
     testCodahaleOperations()
+    testMimeTypes()
     testLiveResources()
     testLiveContainers();
     testRESTModel()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
index dd23926..f90e1ad 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
@@ -18,18 +18,18 @@
 
 package org.apache.slider.agent.rest
 
-import com.sun.jersey.api.client.Client
+import com.sun.jersey.api.client.ClientResponse
 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.http.entity.ContentType
 import org.apache.slider.api.StateValues
 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.registry.docstore.PublishedConfiguration
 import org.apache.slider.core.restclient.HttpOperationResponse
 import org.apache.slider.core.restclient.HttpVerb
 import org.apache.slider.core.restclient.UrlConnectionOperations
@@ -40,6 +40,7 @@ import org.apache.slider.test.Outcome
 import org.apache.slider.test.SliderTestUtils
 
 import javax.ws.rs.core.MediaType
+import java.nio.charset.Charset
 
 import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES
 import static org.apache.slider.api.StatusKeys.*
@@ -80,6 +81,18 @@ class RestTestDelegates extends SliderTestUtils {
     log.info getWebPage(appmaster, SYSTEM_METRICS)
   }
 
+
+  public void testMimeTypes() throws Throwable {
+    describe "Mime Types"
+    HttpOperationResponse response= executeGet(
+        appendToURL(appmaster,
+        SLIDER_PATH_APPLICATION, LIVE_RESOURCES))
+    response.headers.each { key, val -> log.info("$key $val")}
+    log.info "Content type: ${response.contentType}"
+    assert response.contentType.contains(MediaType.APPLICATION_JSON_TYPE.toString())
+  }
+
+  
   public void testLiveResources() throws Throwable {
     describe "Live Resources"
     ConfTreeOperations tree = fetchConfigTree(appmaster, LIVE_RESOURCES)
@@ -140,7 +153,7 @@ class RestTestDelegates extends SliderTestUtils {
     Map<String, SerializedComponentInformation> components =
         fetchType(HashMap, appmaster, LIVE_COMPONENTS)
     // two components
-    assert components.size() == 1
+    assert components.size() >= 1
     log.info "${components}"
 
     SerializedComponentInformation amComponentInfo =
@@ -301,6 +314,7 @@ class RestTestDelegates extends SliderTestUtils {
   public void testSuiteGetOperations() {
 
     testCodahaleOperations()
+    testMimeTypes()
     testLiveResources()
     testLiveContainers();
     testRESTModel()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/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 9cea46c..0abf264 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
@@ -18,20 +18,28 @@
 
 package org.apache.slider.agent.rest
 
+import com.sun.jersey.api.client.Client
+import com.sun.jersey.api.client.config.ClientConfig
+import com.sun.jersey.api.json.JSONConfiguration
+import com.sun.jersey.client.apache.ApacheHttpClient
+import com.sun.jersey.client.apache.ApacheHttpClientHandler
+import com.sun.jersey.client.apache.config.DefaultApacheHttpClientConfig
 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.yarn.api.records.ApplicationReport
 import org.apache.slider.agent.AgentMiniClusterTestBase
+import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
-import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource
-import org.apache.slider.client.SliderClient
 import org.apache.slider.core.main.ServiceLauncher
-
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+import org.apache.slider.core.restclient.HttpOperationResponse
 import org.junit.Test
 
-import static org.apache.slider.server.appmaster.management.MetricsKeys.*
+import static org.apache.slider.server.appmaster.management.MetricsKeys.METRICS_LOGGING_ENABLED
+import static org.apache.slider.server.appmaster.management.MetricsKeys.METRICS_LOGGING_LOG_INTERVAL
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*
 
 @CompileStatic
 @Slf4j
@@ -59,8 +67,9 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     addToTeardown(client);
 
     ApplicationReport report = waitForClusterLive(client)
+    def proxyAM = report.trackingUrl
     def directAM = report.originalTrackingUrl
-
+    
     // set up url config to match
     initHttpTestSupport(launcher.configuration)
 
@@ -73,9 +82,8 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
       def metrics = GET(directAM, SYSTEM_METRICS)
       log.info metrics
     }
-    
-    def proxyAM = report.trackingUrl
 
+    
     GET(proxyAM)
 
     log.info GET(proxyAM, SYSTEM_PING)
@@ -87,33 +95,88 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
         SliderXmlConfKeys.X_DEV_INSECURE_WS,
         true)
 
+    describe "Direct response headers from AM Web resources"
+    def liveResUrl = appendToURL(directAM,
+        SLIDER_PATH_APPLICATION, LIVE_RESOURCES);
+    HttpOperationResponse response = executeGet(liveResUrl)
+    response.headers.each { key, val -> log.info("$key $val") }
+    log.info "Content type: ${response.contentType}"
+
+    describe "proxied response headers from AM Web resources"
+    response = executeGet(appendToURL(proxyAM,
+        SLIDER_PATH_APPLICATION, LIVE_RESOURCES))
+    response.headers.each { key, val -> log.info("$key $val") }
+    log.info "Content type: ${response.contentType}"
 
+    
+    
+    describe "Proxy Jersey Tests"
     JerseyTestDelegates proxyJerseyTests =
-        new JerseyTestDelegates(proxyAM, createJerseyClient())
+        new JerseyTestDelegates(proxyAM, createUGIJerseyClient())
     proxyJerseyTests.testSuiteGetOperations()
 
+    describe "Direct Jersey Tests"
+
     JerseyTestDelegates directJerseyTests =
-        new JerseyTestDelegates(directAM, createJerseyClient())
+        new JerseyTestDelegates(directAM, createUGIJerseyClient())
     directJerseyTests.testSuiteGetOperations()
     directJerseyTests.testSuiteComplexVerbs()
-    
-    RestTestDelegates proxied = new RestTestDelegates(proxyAM)
+
+    describe "Direct Tests"
+
     RestTestDelegates direct = new RestTestDelegates(directAM)
-    
-    proxied.testSuiteGetOperations()
     direct.testSuiteGetOperations()
     direct.testSuiteComplexVerbs()
+
+    describe "Proxy Tests"
+
+    RestTestDelegates proxied = new RestTestDelegates(proxyAM)
+    proxied.testSuiteGetOperations()
     if (!wsBackDoorRequired) {
       // and via the proxy
       proxied.testSuiteComplexVerbs()
     }
     
+
+/*    DISABLED: this client does not pass the tests.
+    
+    // http client direct
+    describe "Proxied Jersey Apache HttpClient"
+    JerseyTestDelegates proxiedHttpClientJersey =
+        new JerseyTestDelegates(proxyAM, createJerseyClientHttpClient())
+    proxiedHttpClientJersey.testSuiteGetOperations()
+    
+    describe "Direct Jersey Apache HttpClient"
+    JerseyTestDelegates directHttpClientJersey =
+        new JerseyTestDelegates(directAM, createJerseyClientHttpClient())
+    directHttpClientJersey.testSuiteGetOperations()
+    directHttpClientJersey.testSuiteComplexVerbs()
+    */
+    createJerseyClientHttpClient()
+    // log the metrics to show what's up
     direct.logCodahaleMetrics();
 
     // this MUST be the final test
     direct.testStop();
   }
 
+  /**
+   * Create Jersey client with URL handling by way
+   * of the Apache HttpClient classes. 
+   * @return a Jersey client
+   */
+  public static Client createJerseyClientHttpClient() {
 
+    def httpclient = new HttpClient(new MultiThreadedHttpConnectionManager());
+    httpclient.httpConnectionManager.params.connectionTimeout = 10000;
+    ClientConfig clientConfig = new DefaultApacheHttpClientConfig();
+    clientConfig.features[JSONConfiguration.FEATURE_POJO_MAPPING] = Boolean.TRUE;
+
+    def handler = new ApacheHttpClientHandler(httpclient, clientConfig);
+
+    def client = new ApacheHttpClient(handler)
+    client.followRedirects = true
+    return client;
+  }
  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
index 897d7e3..1ed340f 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
@@ -22,6 +22,7 @@ import com.sun.jersey.api.client.Client
 import com.sun.jersey.api.client.config.ClientConfig
 import com.sun.jersey.api.client.config.DefaultClientConfig
 import com.sun.jersey.api.json.JSONConfiguration
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler
 import groovy.json.JsonOutput
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
@@ -34,11 +35,11 @@ import org.apache.hadoop.fs.FileStatus
 import org.apache.hadoop.fs.FileSystem as HadoopFS
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.registry.client.types.ServiceRecord
 import org.apache.hadoop.service.ServiceStateException
 import org.apache.hadoop.util.Shell
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.registry.client.types.ServiceRecord
 import org.apache.hadoop.yarn.webapp.ForbiddenException
 import org.apache.hadoop.yarn.webapp.NotFoundException
 import org.apache.slider.api.ClusterDescription
@@ -59,6 +60,7 @@ import org.apache.slider.core.main.ServiceLaunchException
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.core.persist.JsonSerDeser
 import org.apache.slider.core.registry.docstore.PublishedConfigSet
+import org.apache.slider.core.restclient.HttpOperationResponse
 import org.apache.slider.core.restclient.UgiJerseyBinding
 import org.apache.slider.core.restclient.UrlConnectionOperations
 import org.apache.slider.server.appmaster.web.HttpCacheHeaders
@@ -616,17 +618,41 @@ class SliderTestUtils extends Assert {
   }
   
   /**
-   * Create Jersey client
+   * Create Jersey client with UGI integration
    * @return
    */
-  public static Client createJerseyClient() {
+  public static Client createUGIJerseyClient() {
     assertHttpSupportInitialized()
-    ClientConfig clientConfig = new DefaultClientConfig();
-    clientConfig.features[JSONConfiguration.FEATURE_POJO_MAPPING] =Boolean.TRUE;
+    ClientConfig clientConfig = createJerseyClientConfig()
     return new Client(jerseyBinding.handler, clientConfig);
   }
 
   /**
+   * Create Jersey client with URL handling by way
+   * of the java.net classes. This DOES NOT have any SPNEGO
+   * integration. If used to query a secure cluster via the
+   * RM Proxy, it MUST fail.
+   * @return a basic Jersey client
+   */
+  public static Client createBasicJerseyClient() {
+    ClientConfig clientConfig = createJerseyClientConfig()
+    return new Client(new URLConnectionClientHandler(),
+        clientConfig);
+  }
+
+
+  /**
+   * Create a jersey client config with the settings needed for tests
+   * (e.g. POJO mappings)
+   * @return a client config
+   */
+  public static ClientConfig createJerseyClientConfig() {
+    ClientConfig clientConfig = new DefaultClientConfig();
+    clientConfig.features[JSONConfiguration.FEATURE_POJO_MAPPING] = Boolean.TRUE;
+    return clientConfig
+  }
+
+  /**
    * Fetches a web page asserting that the response code is between 200 and 400.
    * Will error on 400 and 500 series response codes and let 200 and 300 through.
    * 
@@ -640,14 +666,18 @@ class SliderTestUtils extends Assert {
    * @return body of response
    */
   public static String getWebPage(String path, Closure connectionChecks = null) {
+    HttpOperationResponse outcome = executeGet(path)
+    return new String(outcome.data);
+  }
+
+  public static HttpOperationResponse executeGet(String path) {
     assert path
     assertHttpSupportInitialized()
 
     log.info("Fetching HTTP content at $path");
     URL url = new URL(path)
     def outcome = connectionOperations.execGet(url)
-    String body = new String(outcome.data)
-    return body;
+    return outcome
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0097ce92/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 95352d0..2d1c863 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
@@ -20,6 +20,9 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+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.common.SliderExitCodes
 import org.apache.slider.common.SliderXmlConfKeys
@@ -72,7 +75,7 @@ public class AgentWebPagesIT extends AgentCommandTestBase
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,
         APP_TEMPLATE,
         APP_RESOURCE2,
-        [Arguments.ARG_OPTION,
+        [ARG_OPTION,
          RestTestDelegates.TEST_GLOBAL_OPTION,
          RestTestDelegates.TEST_GLOBAL_OPTION_PRESENT],
         launchReportFile)
@@ -92,34 +95,55 @@ public class AgentWebPagesIT extends AgentCommandTestBase
     def report = loadAppReport(liveReportFile)
     assert report.url
 
-    def appmaster = report.url
+    def proxyAM = report.url
 
     // get the root page, 
-    getWebPage(appmaster)
+    getWebPage(proxyAM)
     
-    def realappmaster = report.origTrackingUrl;
+    def directAM = report.origTrackingUrl;
     // now attempt direct-to-AM pings
-    RestTestDelegates proxied = new RestTestDelegates(appmaster)
-    RestTestDelegates direct = new RestTestDelegates(realappmaster)
+    RestTestDelegates direct = new RestTestDelegates(directAM)
 
-    proxied.testCodahaleOperations()
-    direct.testCodahaleOperations()
-    proxied.testLiveResources()
+    direct.testSuiteGetOperations()
+    direct.testSuiteComplexVerbs()
 
-    proxied.testRESTModel()
+    // and via the proxy
+    RestTestDelegates proxied = new RestTestDelegates(proxyAM)
+    proxied.testSuiteGetOperations()
+    if (!wsBackDoorRequired) {
+      proxied.testSuiteComplexVerbs()
+    }
+    proxied.logCodahaleMetrics();
 
-    direct.testRestletGetOperations()
-    proxied.testRestletGetOperations()
+    describe "Proxy Jersey Tests"
 
-    // PUT & POST &c direct
-    direct.testPing()
-    if (!wsBackDoorRequired) {
-      // and via the proxy
-      proxied.testRESTModel()
+    JerseyTestDelegates proxyJerseyTests =
+        new JerseyTestDelegates(proxyAM, createUGIJerseyClient())
+    proxyJerseyTests.testSuiteGetOperations()
+
+    describe "Direct Jersey Tests"
+    JerseyTestDelegates directJerseyTests =
+        new JerseyTestDelegates(directAM, createUGIJerseyClient())
+    directJerseyTests.testSuiteGetOperations()
+    directJerseyTests.testSuiteComplexVerbs()
+    
+    if (UserGroupInformation.securityEnabled) {
+      describe "Insecure Proxy Tests against a secure cluster"
+
+      try {
+        String rootpage = fetchWebPageRaisedErrorCodes(proxyAM);
+        fail(" expected a 401, got $rootpage")
+      } catch (ForbiddenException expected) {
+        // expected
+      }
+      
+      // these tests use the Jersey client without the Hadoop-specific
+      // SPNEGO
+      JerseyTestDelegates baseicJerseyClientTests =
+          new JerseyTestDelegates(proxyAM, createBasicJerseyClient())
+      baseicJerseyClientTests.testSuiteGetOperations()
     }
     
-    direct.logCodahaleMetrics();
-
     // finally, stop the AM
     direct.testStop();
   }