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/09 16:34:27 UTC

[06/10] incubator-slider git commit: SLIDER-719 working on IT tests for PUT/POST/DELETE/HEAD. This is failing as POST is being 302'd

SLIDER-719 working on IT tests for PUT/POST/DELETE/HEAD. This is failing as POST is being 302'd


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

Branch: refs/heads/develop
Commit: f7696615df5e0e640813f8b6ffee319a7b2b870f
Parents: 95d4acb
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 9 13:52:06 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 9 13:52:19 2015 +0000

----------------------------------------------------------------------
 .../launch/SerializedApplicationReport.java     |   2 +-
 .../slider/agent/rest/RestTestDelegates.groovy  | 235 +++++++++++++++
 .../slider/agent/rest/TestStandaloneREST.groovy | 109 +++++++
 .../standalone/TestStandaloneAgentWeb.groovy    | 286 -------------------
 .../apache/slider/test/SliderTestBase.groovy    |   4 +-
 .../apache/slider/test/SliderTestUtils.groovy   |  18 +-
 .../funtest/lifecycle/AgentWebPagesIT.groovy    |  40 ++-
 7 files changed, 393 insertions(+), 301 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f7696615/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java b/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
index c1880b5..377c87a 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
@@ -82,7 +82,7 @@ public class SerializedApplicationReport {
     this.finalStatus = appStatus == null ? "" : appStatus.toString();
     this.progress = report.getProgress();
     this.url = report.getTrackingUrl();
-    this.origTrackingUrl= report.getTrackingUrl();
+    this.origTrackingUrl= report.getOriginalTrackingUrl();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f7696615/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
new file mode 100644
index 0000000..f5264b4
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/RestTestDelegates.groovy
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.agent.rest
+
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.webapp.NotFoundException
+import org.apache.slider.api.StateValues
+import org.apache.slider.api.types.SerializedComponentInformation
+import org.apache.slider.api.types.SerializedContainerInformation
+import org.apache.slider.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.core.restclient.UrlConnectionOperations
+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 org.junit.Test
+
+import javax.ws.rs.core.MediaType
+
+import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES
+import static org.apache.slider.api.StatusKeys.*
+import static org.apache.slider.common.SliderKeys.COMPONENT_AM
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*
+
+/**
+ * This class contains parts of tests that can be run
+ * against a deployed AM: local or remote
+ */
+@CompileStatic
+@Slf4j
+class RestTestDelegates extends SliderTestUtils {
+  public static final String TEST_GLOBAL_OPTION = "test.global.option"
+  public static final String TEST_GLOBAL_OPTION_PRESENT = "present"
+
+  final String appmaster;
+
+  RestTestDelegates(String appmaster) {
+    this.appmaster = appmaster
+  }
+
+  public void testCodahaleOperations() throws Throwable {
+    describe "Codahale operations"
+    // now switch to the Hadoop URL connection, with SPNEGO escalation
+    getWebPage(appmaster)
+    getWebPage(appmaster, SYSTEM_THREADS)
+    getWebPage(appmaster, SYSTEM_HEALTHCHECK)
+    getWebPage(appmaster, SYSTEM_METRICS_JSON)
+  }
+
+  @Test
+  public void testLiveResources() throws Throwable {
+    describe "Live Resources"
+    ConfTreeOperations tree = fetchConfigTree(appmaster, LIVE_RESOURCES)
+
+    log.info tree.toString()
+    def liveAM = tree.getComponent(COMPONENT_AM)
+    def desiredInstances = liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES);
+    assert desiredInstances ==
+           liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_ACTUAL)
+
+    assert 1 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_STARTED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_REQUESTING)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_FAILED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_COMPLETED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_RELEASING)
+  }
+
+  @Test
+  public void testLiveContainers() throws Throwable {
+    describe "Application REST ${LIVE_CONTAINERS}"
+
+    Map<String, SerializedContainerInformation> containers =
+        fetchType(HashMap, appmaster, LIVE_CONTAINERS)
+    assert containers.size() == 1
+    log.info "${containers}"
+    SerializedContainerInformation amContainerInfo =
+        (SerializedContainerInformation) containers.values()[0]
+    assert amContainerInfo.containerId
+
+    def amContainerId = amContainerInfo.containerId
+    assert containers[amContainerId]
+
+    assert amContainerInfo.component == COMPONENT_AM
+    assert amContainerInfo.createTime > 0
+    assert amContainerInfo.exitCode == null
+    assert amContainerInfo.output == null
+    assert amContainerInfo.released == null
+    assert amContainerInfo.state == StateValues.STATE_LIVE
+
+    describe "containers"
+
+    SerializedContainerInformation retrievedContainerInfo =
+        fetchType(SerializedContainerInformation, appmaster,
+            LIVE_CONTAINERS + "/${amContainerId}")
+    assert retrievedContainerInfo.containerId == amContainerId
+
+    // fetch missing
+    try {
+      def result = fetchType(SerializedContainerInformation, appmaster,
+          LIVE_CONTAINERS + "/unknown")
+      fail("expected an error, got $result")
+    } catch (NotFoundException e) {
+      // expected
+    }
+
+
+    describe "components"
+
+    Map<String, SerializedComponentInformation> components =
+        fetchType(HashMap, appmaster, LIVE_COMPONENTS)
+    // two components
+    assert components.size() == 1
+    log.info "${components}"
+
+    SerializedComponentInformation amComponentInfo =
+        (SerializedComponentInformation) components[COMPONENT_AM]
+
+    SerializedComponentInformation amFullInfo = fetchType(
+        SerializedComponentInformation,
+        appmaster,
+        LIVE_COMPONENTS + "/${COMPONENT_AM}")
+
+    assert amFullInfo.containers.size() == 1
+    assert amFullInfo.containers[0] == amContainerId
+
+  }
+
+  /**
+   * Test the rest model. For this to work the cluster has to be configured
+   * with the global option
+   * @param appmaster
+   */
+  public void testRESTModel(String appmaster) {
+    describe "model"
+
+    assertPathServesList(appmaster,
+        MODEL,
+        ApplicationResource.MODEL_ENTRIES)
+
+    def unresolvedConf = fetchType(AggregateConf, appmaster, MODEL_DESIRED)
+//    log.info "Unresolved \n$unresolvedConf"
+    def unresolvedAppConf = unresolvedConf.appConfOperations
+
+    def sam = "slider-appmaster"
+    assert unresolvedAppConf.getComponentOpt(sam,
+        TEST_GLOBAL_OPTION, "") == ""
+    def resolvedConf = fetchType(AggregateConf, appmaster, MODEL_RESOLVED)
+//    log.info "Resolved \n$resolvedConf"
+    assert resolvedConf.appConfOperations.getComponentOpt(
+        sam, TEST_GLOBAL_OPTION, "") == TEST_GLOBAL_OPTION_PRESENT
+
+    def unresolved = fetchTypeList(ConfTree, appmaster,
+        [MODEL_DESIRED_APPCONF, MODEL_DESIRED_RESOURCES])
+    assert unresolved[MODEL_DESIRED_APPCONF].components[sam]
+    [TEST_GLOBAL_OPTION] == null
+
+
+    def resolved = fetchTypeList(ConfTree, appmaster,
+        [MODEL_RESOLVED_APPCONF, MODEL_RESOLVED_RESOURCES])
+    assert resolved[MODEL_RESOLVED_APPCONF].components[sam]
+    [TEST_GLOBAL_OPTION] ==
+    TEST_GLOBAL_OPTION_PRESENT
+  }
+
+  public void testPing(String appmaster) {
+    // GET
+    String ping = appendToURL(appmaster, SLIDER_PATH_APPLICATION, ACTION_PING)
+    describe "ping to AM URL $appmaster, ping URL $ping"
+    def pinged = fetchType(PingResource, appmaster, ACTION_PING + "?body=hello")
+    log.info "Ping GET: $pinged"
+
+    URL pingUrl = new URL(ping)
+    def message = "hello"
+
+    // HEAD
+    pingAction(HttpVerb.HEAD, pingUrl, message)
+
+    // Other verbs
+    pingAction(HttpVerb.POST, pingUrl, message)
+    pingAction(HttpVerb.PUT, pingUrl, message)
+    pingAction(HttpVerb.DELETE, pingUrl, message)
+
+  }
+
+
+  public HttpOperationResponse pingAction(
+      HttpVerb verb,
+      URL pingUrl,
+      String payload) {
+    return pingAction(connectionFactory, verb, pingUrl, payload)
+  }
+
+  public HttpOperationResponse pingAction(
+      UrlConnectionOperations ops, HttpVerb verb, URL pingUrl, String payload) {
+    def pinged
+    def outcome = ops.execHttpOperation(
+        verb,
+        pingUrl,
+        payload.bytes,
+        MediaType.TEXT_PLAIN)
+    byte[] bytes = outcome.data
+    if (verb.hasResponseBody()) {
+      assert bytes.length > 0, "0 bytes from ping $verb.verb"
+      pinged = deser(PingResource, bytes)
+      log.info "Ping $verb.verb: $pinged"
+      assert verb.verb == pinged.verb
+    } else {
+      assert bytes.length ==
+             0, "${bytes.length} bytes of data from ping $verb.verb"
+    }
+    return outcome
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f7696615/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
new file mode 100644
index 0000000..582ddc7
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.agent.rest
+
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.slider.agent.AgentMiniClusterTestBase
+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.junit.Test
+
+import static org.apache.slider.server.appmaster.management.MetricsKeys.*
+
+@CompileStatic
+@Slf4j
+class TestStandaloneREST extends AgentMiniClusterTestBase {
+
+
+  @Test
+  public void testStandaloneAgentWeb() throws Throwable {
+
+    describe "create a standalone AM then perform actions on it"
+    //launch fake master
+    def conf = configuration
+    conf.setBoolean(METRICS_LOGGING_ENABLED, true)
+    conf.setInt(METRICS_LOGGING_LOG_INTERVAL, 1)
+    String clustername = createMiniCluster("", conf, 1, true)
+
+
+    ServiceLauncher<SliderClient> launcher =
+        createStandaloneAMWithArgs(clustername,
+            [Arguments.ARG_OPTION,
+             RestTestDelegates.TEST_GLOBAL_OPTION, 
+             RestTestDelegates.TEST_GLOBAL_OPTION_PRESENT],
+            true, false)
+    SliderClient client = launcher.service
+    addToTeardown(client);
+
+    ApplicationReport report = waitForClusterLive(client)
+    def realappmaster = report.originalTrackingUrl
+
+    // set up url config to match
+    initConnectionFactory(launcher.configuration)
+
+
+    execHttpRequest(WEB_STARTUP_TIME) {
+      GET(realappmaster)
+    }
+    
+    execHttpRequest(WEB_STARTUP_TIME) {
+      def metrics = GET(realappmaster, SYSTEM_METRICS)
+      log.info metrics
+    }
+    
+    sleep(5000)
+    def appmaster = report.trackingUrl
+
+    GET(appmaster)
+
+    log.info GET(appmaster, SYSTEM_PING)
+    log.info GET(appmaster, SYSTEM_THREADS)
+    log.info GET(appmaster, SYSTEM_HEALTHCHECK)
+    log.info GET(appmaster, SYSTEM_METRICS_JSON)
+
+    RestTestDelegates proxied = new RestTestDelegates(appmaster)
+    RestTestDelegates direct = new RestTestDelegates(realappmaster)
+    
+    proxied.testCodahaleOperations()
+    direct.testCodahaleOperations()
+
+    describe "base entry lists"
+
+    assertPathServesList(appmaster, LIVE, ApplicationResource.LIVE_ENTRIES)
+
+    // now some REST gets
+    describe "Application REST ${LIVE_RESOURCES}"
+    proxied.testLiveResources()
+
+    proxied.testRESTModel(appmaster)
+    
+    // PUT & POST &c must go direct for now
+    direct.testPing(realappmaster)
+
+  }
+
+
+ 
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f7696615/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
deleted file mode 100644
index d5be646..0000000
--- a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
+++ /dev/null
@@ -1,286 +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.agent.standalone
-
-import groovy.transform.CompileStatic
-import groovy.util.logging.Slf4j
-import org.apache.hadoop.yarn.api.records.ApplicationReport
-import org.apache.hadoop.yarn.webapp.NotFoundException
-import org.apache.slider.agent.AgentMiniClusterTestBase
-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.params.Arguments
-import org.apache.slider.core.conf.AggregateConf
-import org.apache.slider.core.conf.ConfTree
-import org.apache.slider.core.restclient.HttpOperationResponse
-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 javax.ws.rs.core.MediaType
-
-import static org.apache.slider.api.ResourceKeys.*
-import static org.apache.slider.api.StatusKeys.*
-import org.apache.slider.client.SliderClient
-import static org.apache.slider.common.SliderKeys.*;
-import org.apache.slider.core.conf.ConfTreeOperations
-import org.apache.slider.core.main.ServiceLauncher
-
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
-import org.junit.Test
-
-import static org.apache.slider.server.appmaster.management.MetricsKeys.*
-
-@CompileStatic
-@Slf4j
-class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
-  
-  public static final int WEB_STARTUP_TIME = 30000
-  public static final String TEST_GLOBAL_OPTION = "test.global.option"
-  public static final String TEST_GLOBAL_OPTION_PRESENT = "present"
-  public static final byte[] NO_BYTES = new byte[0]
-
-  @Test
-  public void testStandaloneAgentWeb() throws Throwable {
-
-    describe "create a standalone AM then perform actions on it"
-    //launch fake master
-    def conf = configuration
-    conf.setBoolean(METRICS_LOGGING_ENABLED, true)
-    conf.setInt(METRICS_LOGGING_LOG_INTERVAL, 1)
-    String clustername = createMiniCluster("", conf, 1, true)
-
-
-    ServiceLauncher<SliderClient> launcher =
-        createStandaloneAMWithArgs(clustername,
-            [Arguments.ARG_OPTION,
-             TEST_GLOBAL_OPTION, TEST_GLOBAL_OPTION_PRESENT],
-            true, false)
-    SliderClient client = launcher.service
-    addToTeardown(client);
-
-    ApplicationReport report = waitForClusterLive(client)
-    def realappmaster = report.originalTrackingUrl
-
-    // set up url config to match
-    initConnectionFactory(launcher.configuration)
-
-
-    execHttpRequest(WEB_STARTUP_TIME) {
-      GET(realappmaster)
-    }
-    
-    execHttpRequest(WEB_STARTUP_TIME) {
-      def metrics = GET(realappmaster, SYSTEM_METRICS)
-      log.info metrics
-    }
-    
-    sleep(5000)
-    def appmaster = report.trackingUrl
-
-    GET(appmaster)
-
-    log.info GET(appmaster, SYSTEM_PING)
-    log.info GET(appmaster, SYSTEM_THREADS)
-    log.info GET(appmaster, SYSTEM_HEALTHCHECK)
-    log.info GET(appmaster, SYSTEM_METRICS_JSON)
-    
-    describe "Codahale operations"
-    // now switch to the Hadoop URL connection, with SPNEGO escalation
-    getWebPage(appmaster)
-    getWebPage(appmaster, SYSTEM_THREADS)
-    getWebPage(appmaster, SYSTEM_HEALTHCHECK)
-    getWebPage(appmaster, SYSTEM_METRICS_JSON)
-    
-    log.info getWebPage(realappmaster, SYSTEM_METRICS_JSON)
-
-    // get the root page, including some checks for cache disabled
-    getWebPage(appmaster, {
-      HttpURLConnection conn ->
-        assertConnectionNotCaching(conn)
-    })
-
-    // now some REST gets
-    describe "Application REST ${LIVE_RESOURCES}"
-
-    ConfTreeOperations tree = fetchConfigTree(conf, appmaster, LIVE_RESOURCES)
-
-    log.info tree.toString()
-    def liveAM = tree.getComponent(COMPONENT_AM)
-    def desiredInstances = liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES);
-    assert desiredInstances == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_ACTUAL)
-
-    assert 1 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_STARTED)
-    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_REQUESTING)
-    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_FAILED)
-    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_COMPLETED)
-    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_RELEASING)
-
-    describe "Application REST ${LIVE_CONTAINERS}"
-
-    Map<String, SerializedContainerInformation> containers =
-        fetchType(HashMap, appmaster, LIVE_CONTAINERS)
-    assert containers.size() == 1
-    log.info "${containers}"
-    SerializedContainerInformation amContainerInfo = (SerializedContainerInformation) containers.values()[0]
-    assert amContainerInfo.containerId
-
-    def amContainerId = amContainerInfo.containerId
-    assert containers[amContainerId]
-
-    assert amContainerInfo.component == COMPONENT_AM
-    assert amContainerInfo.createTime > 0
-    assert amContainerInfo.exitCode == null
-    assert amContainerInfo.output == null
-    assert amContainerInfo.released == null
-    assert amContainerInfo.state == StateValues.STATE_LIVE
-   
-    describe "base entry lists"
-
-    assertPathServesList(appmaster, LIVE, ApplicationResource.LIVE_ENTRIES)
-    
-    describe "containers"
-
-    SerializedContainerInformation retrievedContainerInfo =
-        fetchType(SerializedContainerInformation, appmaster,
-            LIVE_CONTAINERS +"/${amContainerId}")
-    assert retrievedContainerInfo.containerId == amContainerId
-    
-    // fetch missing
-    try {
-      def result = fetchType(SerializedContainerInformation, appmaster,
-          LIVE_CONTAINERS + "/unknown")
-      fail("expected an error, got $result")
-    } catch (NotFoundException e) {
-      // expected
-    }
-
-    describe "components"
-
-    Map<String, SerializedComponentInformation> components =
-        fetchType(HashMap, appmaster, LIVE_COMPONENTS)
-    // two components
-    assert components.size() == 1
-    log.info "${components}"
-
-    SerializedComponentInformation amComponentInfo =
-        (SerializedComponentInformation)components[COMPONENT_AM]
-
-    SerializedComponentInformation amFullInfo = fetchType(
-        SerializedComponentInformation,
-        appmaster,
-        LIVE_COMPONENTS +"/${COMPONENT_AM}")
-
-    assert amFullInfo.containers.size() == 1
-    assert amFullInfo.containers[0] == amContainerId
-
-    testRESTModel(appmaster)
-    
-    // PUT & POST &c must go direct for now
-    String wsroot = appendToURL(realappmaster, SLIDER_CONTEXT_ROOT)
-    testPing(realappmaster)
-
-  }
-
-  public void testRESTModel(String appmaster) {
-    describe "model"
-
-    assertPathServesList(appmaster,
-        MODEL,
-        ApplicationResource.MODEL_ENTRIES)
-
-    def unresolvedConf = fetchType(AggregateConf, appmaster, MODEL_DESIRED)
-//    log.info "Unresolved \n$unresolvedConf"
-    def unresolvedAppConf = unresolvedConf.appConfOperations
-
-    def sam = "slider-appmaster"
-    assert unresolvedAppConf.getComponentOpt(sam,
-        TEST_GLOBAL_OPTION, "") == ""
-    def resolvedConf = fetchType(AggregateConf, appmaster, MODEL_RESOLVED)
-//    log.info "Resolved \n$resolvedConf"
-    assert resolvedConf.appConfOperations.getComponentOpt(
-        sam, TEST_GLOBAL_OPTION, "") == TEST_GLOBAL_OPTION_PRESENT
-
-    def unresolved = fetchTypeList(ConfTree, appmaster,
-        [MODEL_DESIRED_APPCONF, MODEL_DESIRED_RESOURCES])
-    assert unresolved[MODEL_DESIRED_APPCONF].components[sam][TEST_GLOBAL_OPTION] == null
-
-
-    def resolved = fetchTypeList(ConfTree, appmaster,
-        [MODEL_RESOLVED_APPCONF, MODEL_RESOLVED_RESOURCES])
-    assert resolved[MODEL_RESOLVED_APPCONF].components[sam][TEST_GLOBAL_OPTION] ==
-           TEST_GLOBAL_OPTION_PRESENT
-  }
-
-  /**
-   * 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 = fetchType(ArrayList, appmaster, path)
-    assert list.size() == entries.size()
-    assert entries.containsAll(list)
-  }
-
-  public void testPing(String appmaster) {
-    // GET
-    String ping = appendToURL(appmaster, SLIDER_PATH_APPLICATION, ACTION_PING)
-    describe "ping to AM URL $appmaster, ping URL $ping"
-    def pinged = fetchType(PingResource, appmaster,  ACTION_PING +"?body=hello")
-    log.info "Ping GET: $pinged"
-    
-    // POST
-    URL pingUrl = new URL(ping)
-
-
-    def message = "hello"
-    pingAction(HttpVerb.POST, pingUrl, message)
-    pingAction(HttpVerb.PUT, pingUrl, message)
-    pingAction(HttpVerb.DELETE, pingUrl, message)
-    pingAction(HttpVerb.HEAD, pingUrl, message)
-
-  }
-
-  public HttpOperationResponse pingAction(HttpVerb verb, URL pingUrl, String payload) {
-    def pinged
-    def outcome = connectionFactory.execHttpOperation(
-        verb,
-        pingUrl,
-        payload.bytes,
-        MediaType.TEXT_PLAIN)
-    byte[] bytes = outcome.data
-    if (verb.hasResponseBody()) {
-      assert bytes.length > 0, "0 bytes from ping $verb.verb"
-      pinged = deser(PingResource, bytes)
-      log.info "Ping $verb.verb: $pinged"
-      assert verb.verb == pinged.verb
-    } else {
-      assert bytes.length == 0, "${bytes.length} bytes of data from ping $verb.verb"
-    }
-    return outcome
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f7696615/slider-core/src/test/groovy/org/apache/slider/test/SliderTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestBase.groovy b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestBase.groovy
index 7be7869..7a935dc 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestBase.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestBase.groovy
@@ -43,7 +43,9 @@ public abstract class SliderTestBase extends SliderTestUtils {
    * Singleton metric registry
    */
   public static final MetricsAndMonitoring metrics = new MetricsAndMonitoring()
-  
+  public static final int WEB_STARTUP_TIME = 30000
+  public static final byte[] NO_BYTES = new byte[0]
+
   @Rule
   public TestName methodName = new TestName();
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f7696615/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 87e3206..5034ba6 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
@@ -1173,7 +1173,7 @@ class SliderTestUtils extends Assert {
   }
   
   public ConfTreeOperations fetchConfigTree(
-      YarnConfiguration conf, String appmaster, String subpath) {
+      String appmaster, String subpath) {
     ConfTree ctree = fetchType(ConfTree, appmaster, subpath)
     ConfTreeOperations tree = new ConfTreeOperations(ctree)
     return tree
@@ -1195,4 +1195,20 @@ class SliderTestUtils extends Assert {
     }
     return results;
   }
+
+  /**
+   * 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 = fetchType(ArrayList, appmaster, path)
+    assert list.size() == entries.size()
+    assert entries.containsAll(list)
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f7696615/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 e5d5f2a..dcfc01b 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,7 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.slider.agent.rest.RestTestDelegates
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -60,7 +61,9 @@ public class AgentWebPagesIT extends AgentCommandTestBase
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,
         APP_TEMPLATE,
         APP_RESOURCE2,
-        [],
+        [Arguments.ARG_OPTION,
+         RestTestDelegates.TEST_GLOBAL_OPTION,
+         RestTestDelegates.TEST_GLOBAL_OPTION_PRESENT],
         launchReportFile)
 
     logShell(shell)
@@ -78,17 +81,30 @@ public class AgentWebPagesIT extends AgentCommandTestBase
     def report = loadAppReport(liveReportFile)
     assert report.url
 
-    def root = report.url
-
-    // get the root page, including some checks for cache disabled
-    getWebPage(root, {
-      HttpURLConnection conn ->
-        assertConnectionNotCaching(conn)
-    })
-    log.info getWebPage(root, RestPaths.SYSTEM_METRICS)
-    log.info getWebPage(root, RestPaths.SYSTEM_THREADS)
-    log.info getWebPage(root, RestPaths.SYSTEM_HEALTHCHECK)
-    log.info getWebPage(root, RestPaths.SYSTEM_PING)
+    def appmaster = report.url
+
+    // get the root page, 
+    getWebPage(appmaster)
+    
+    // query Coda Hale metrics
+    log.info getWebPage(appmaster, RestPaths.SYSTEM_METRICS)
+    log.info getWebPage(appmaster, RestPaths.SYSTEM_THREADS)
+    log.info getWebPage(appmaster, RestPaths.SYSTEM_HEALTHCHECK)
+    log.info getWebPage(appmaster, RestPaths.SYSTEM_PING)
+
+    def realappmaster = report.origTrackingUrl;
+    // now attempt direct-to-AM pings
+    RestTestDelegates proxied = new RestTestDelegates(appmaster)
+    RestTestDelegates direct = new RestTestDelegates(realappmaster)
+
+    proxied.testCodahaleOperations()
+    direct.testCodahaleOperations()
+    proxied.testLiveResources()
+
+    proxied.testRESTModel(appmaster)
+
+    // PUT & POST &c must go direct for now
+    direct.testPing(realappmaster)
   }
 
 }