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

[01/10] incubator-slider git commit: SLIDER-531 re-enable registration of IPC endpoint now that hadoop-2.6 only uses the "2nd generation" registry format

Repository: incubator-slider
Updated Branches:
  refs/heads/develop 7a1a590f3 -> aaeac3196


SLIDER-531 re-enable registration of IPC endpoint now that hadoop-2.6 only uses the "2nd generation" registry format


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

Branch: refs/heads/develop
Commit: ceb21e15748df0f66e02c34cb694c32fb6f2a146
Parents: 7a1a590
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 7 12:29:05 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 7 12:29:05 2015 +0000

----------------------------------------------------------------------
 .../org/apache/slider/server/appmaster/SliderAppMaster.java    | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ceb21e15/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 0dd6c59..7b9f6db 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -106,6 +107,7 @@ import org.apache.slider.core.main.LauncherExitCodes;
 import org.apache.slider.core.main.RunService;
 import org.apache.slider.core.main.ServiceLauncher;
 import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.core.registry.info.CustomRegistryConstants;
 import org.apache.slider.providers.ProviderCompleted;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderService;
@@ -1132,15 +1134,11 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
         PersistencePolicies.APPLICATION);
     serviceRecord.description = "Slider Application Master";
 
-/* SLIDER-531: disable this addition so things compile against versions of
-the registry with/without the new record format
-
     serviceRecord.addExternalEndpoint(
         RegistryTypeUtils.ipcEndpoint(
             CustomRegistryConstants.AM_IPC_PROTOCOL,
             rpcServiceAddress));
             
-    */
     // internal services
     sliderAMProvider.applyInitialRegistryDefinitions(amWebURI,
         agentOpsURI,


[08/10] incubator-slider git commit: SLIDER-710 WS/ backdoor is open everywhere; funtests verifying ping operations

Posted by st...@apache.org.
SLIDER-710 WS/ backdoor is open everywhere; funtests verifying ping operations


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

Branch: refs/heads/develop
Commit: d2e8f88a3fcb15aaf56801e3dce00e481c8ae0c1
Parents: 63dc3bb
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 9 15:09:33 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 9 15:09:33 2015 +0000

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  |  2 +
 .../slider/common/tools/ConfigHelper.java       | 40 ++++++++++++++++++++
 .../restclient/UrlConnectionOperations.java     | 14 ++++---
 .../slider/providers/SliderProviderFactory.java |  5 +++
 .../server/appmaster/SliderAppMaster.java       |  7 +++-
 .../appmaster/web/rest/InsecureAmFilter.java    |  6 ++-
 .../web/rest/InsecureAmFilterInitializer.java   |  5 +--
 .../apache/slider/test/SliderTestUtils.groovy   | 19 ++++++----
 .../funtest/framework/CommandTestBase.groovy    |  1 +
 .../funtest/lifecycle/AgentWebPagesIT.groovy    |  5 ++-
 10 files changed, 84 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 2d1af35..59d317e 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -245,6 +245,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     config = super.bindArgs(config, args);
     serviceArgs = new ClientArgs(args);
     serviceArgs.parse();
+    // add the slider XML config
+    ConfigHelper.injectSliderXMLResource();
     // yarn-ify
     YarnConfiguration yarnConfiguration = new YarnConfiguration(config);
     return SliderUtils.patchConfiguration(yarnConfiguration);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java b/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
index c7b79f0..9db241d 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
@@ -51,6 +51,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * Methods to aid in config, both in the Configuration class and
@@ -62,6 +63,11 @@ import java.util.TreeSet;
 public class ConfigHelper {
   private static final Logger log = LoggerFactory.getLogger(ConfigHelper.class);
 
+  private static AtomicBoolean sliderResourceInjected =
+      new AtomicBoolean(false);
+  private static AtomicBoolean sliderResourceInjectionAttempted =
+      new AtomicBoolean(false);
+  
   /**
    * Dump the (sorted) configuration
    * @param conf config
@@ -615,4 +621,38 @@ public class ConfigHelper {
         RegistryConstants.KEY_REGISTRY_ZK_ROOT);
     
   }
+
+  /**
+   * Load a configuration with the {@link SliderKeys#SLIDER_XML} resource
+   * included
+   * @return a configuration instance
+   */
+  public static Configuration loadSliderConfiguration() {
+    Configuration conf = new Configuration();
+    conf.addResource(SliderKeys.SLIDER_XML);
+    return conf;
+  }
+
+  /**
+   * Inject the {@link SliderKeys#SLIDER_XML} resource
+   * into the configuration resources <i>of all configurations</i>.
+   * <p>
+   *   This operation is idempotent.
+   * <p>
+   * If the resource is not on the classpath, downgrades, rather than
+   * fails.
+   * @return true if the resource was found and loaded.
+   */
+  public static synchronized boolean injectSliderXMLResource() {
+    if (sliderResourceInjectionAttempted.getAndSet(true)) {
+      return sliderResourceInjected.get();
+    }
+    URL resourceUrl = getResourceUrl(SliderKeys.SLIDER_XML);
+    if (resourceUrl != null) {
+      Configuration.addDefaultResource(SliderKeys.SLIDER_XML);
+      sliderResourceInjected.set(true);
+    }
+    return sliderResourceInjected.get();
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/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 4633414..fa0da5a 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
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.core.MediaType;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -149,7 +148,7 @@ public class UrlConnectionOperations extends Configured {
         conn.disconnect();
       }
     }
-    uprateFaults(url.toString(), resultCode, body);
+    uprateFaults(HttpVerb.GET, url, resultCode, body);
     outcome.responseCode = resultCode;
     outcome.data = body;
     return outcome;
@@ -159,12 +158,14 @@ public class UrlConnectionOperations extends Configured {
    * Uprate error codes 400 and up into faults; 
    * 404 is converted to a {@link NotFoundException},
    * 401 to {@link ForbiddenException}
+   *
+   * @param verb HTTP Verb used
    * @param url URL as string
    * @param resultCode response from the request
    * @param body optional body of the request
    * @throws IOException if the result was considered a failure
    */
-  public static void uprateFaults(String url,
+  public static void uprateFaults(HttpVerb verb, URL url,
       int resultCode, byte[] body)
       throws IOException {
 
@@ -172,11 +173,12 @@ public class UrlConnectionOperations extends Configured {
       //success
       return;
     }
+    String msg = verb.toString() +" "+ url.toString();
     if (resultCode == 404) {
-      throw new NotFoundException(url);
+      throw new NotFoundException(msg);
     }
     if (resultCode == 401) {
-      throw new ForbiddenException(url);
+      throw new ForbiddenException(msg);
     }
     // all other error codes
     String bodyAsString;
@@ -185,7 +187,7 @@ public class UrlConnectionOperations extends Configured {
     } else {
       bodyAsString = "";
     }
-    String message = "Request to " + url +
+    String message =  msg +
                      " failed with exit code " + resultCode
                      + ", body length " + bodyAsString.length()
                      + ":\n" + bodyAsString;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java b/slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java
index f3bf0b1..5dd4a32 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java
@@ -97,6 +97,11 @@ public abstract class SliderProviderFactory extends Configured {
                               ex);
   }
 
+  /**
+   * Load a configuration with the {@link SliderKeys#SLIDER_XML} resource
+   * included
+   * @return a configuration instance
+   */
   public static Configuration loadSliderConfiguration() {
     Configuration conf = new Configuration();
     conf.addResource(SliderKeys.SLIDER_XML);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index cc9bf82..a6974b9 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -503,6 +503,9 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
                                                                       Exception {
     // let the superclass process it
     Configuration superConf = super.bindArgs(config, args);
+    // add the slider XML config
+    ConfigHelper.injectSliderXMLResource();
+
     //yarn-ify
     YarnConfiguration yarnConfiguration = new YarnConfiguration(
         superConf);
@@ -574,7 +577,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
   /**
    * Create and run the cluster.
-   * @param clustername
+   * @param clustername cluster name
    * @return exit code
    * @throws Throwable on a failure
    */
@@ -2171,7 +2174,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
   @Override //  NMClientAsync.CallbackHandler 
   public void onStartContainerError(ContainerId containerId, Throwable t) {
-    LOG_YARN.error("Failed to start Container " + containerId, t);
+    LOG_YARN.error("Failed to start Container {}", containerId, t);
     appState.onNodeManagerContainerStartFailed(containerId, t);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
index 07b19e7..6ee3d91 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
@@ -51,6 +51,9 @@ public class InsecureAmFilter extends AmIpFilter {
   public void init(FilterConfig conf) throws ServletException {
     super.init(conf);
     wsContextRoot = conf.getInitParameter(WS_CONTEXT_ROOT);
+    if (wsContextRoot == null) {
+      throw new ServletException("No value set for " + WS_CONTEXT_ROOT);
+    }
   }
 
   private void rejectNonHttpRequests(ServletRequest req) throws
@@ -69,7 +72,8 @@ public class InsecureAmFilter extends AmIpFilter {
     HttpServletResponse httpResp = (HttpServletResponse) resp;
 
 
-    if (!httpReq.getRequestURI().startsWith(wsContextRoot)) {
+    String requestURI = httpReq.getRequestURI();
+    if (requestURI == null || !requestURI.startsWith(wsContextRoot)) {
       // hand off to the AM filter if it is not the context root
       super.doFilter(req, resp, chain);
       return;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
index 111d715..42a5bdd 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
@@ -46,7 +46,7 @@ public class InsecureAmFilterInitializer extends FilterInitializer {
   private Configuration configuration;
 
   public static final String NAME =
-      "org.apache.slider.server.appmaster.web.InsecureAmFilterInitializer";
+      "org.apache.slider.server.appmaster.web.rest.InsecureAmFilterInitializer";
 
   @Override
   public void initFilter(FilterContainer container, Configuration conf) {
@@ -59,8 +59,7 @@ public class InsecureAmFilterInitializer extends FilterInitializer {
     params.put(InsecureAmFilter.PROXY_URI_BASE, getHttpSchemePrefix()
                                                 + proxy +
                                                 getApplicationWebProxyBase());
-    params.put(InsecureAmFilter.WS_CONTEXT_ROOT,
-        conf.get(InsecureAmFilter.WS_CONTEXT_ROOT));
+    params.put(InsecureAmFilter.WS_CONTEXT_ROOT, RestPaths.WS_CONTEXT_ROOT);
     container.addFilter(FILTER_NAME, FILTER_CLASS, params);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/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 5034ba6..fca1128 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
@@ -505,17 +505,22 @@ class SliderTestUtils extends Assert {
 
     def body = get.responseBodyAsString
 
-    uprateFaults(url, resultCode, body)
+    uprateFaults("GET", url, resultCode, body)
     return body;
   }
 
   /**
    *  uprate some faults
-   * @param url
-   * @param resultCode
-   * @param body
+   * @param verb HTTP verb
+   * @param url URL
+   * @param resultCode result code
+   * @param body any body
    */
-  public static void uprateFaults(String url, int resultCode, String body) {
+  public static void uprateFaults(
+      String verb,
+      String url,
+      int resultCode,
+      String body) {
 
     if (resultCode == 404) {
       throw new NotFoundException(url);
@@ -524,10 +529,10 @@ class SliderTestUtils extends Assert {
       throw new ForbiddenException(url);
     }
     if (!(resultCode >= 200 && resultCode < 400)) {
-      String message = "Request to " + url +
+      String message = "$verb to $url " +
                        " failed with exit code " +
                        resultCode + ", body length " +
-                       body?.length() + ":\n" + "body;"
+                       body?.length() + ":\n" + body
       log.error(message);
       throw new IOException(message);
     }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 00b4062..4871ef8 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -103,6 +103,7 @@ abstract class CommandTestBase extends SliderTestUtils {
    */
   static {
     new HdfsConfiguration()
+    ConfigHelper.injectSliderXMLResource()
     ConfigHelper.registerDeprecatedConfigItems();
     SLIDER_CONFIG = ConfLoader.loadSliderConf(SLIDER_CONF_XML, true);
     THAW_WAIT_TIME = getTimeOptionMillis(SLIDER_CONFIG,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2e8f88a/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 dcfc01b..c71df8a 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
@@ -22,6 +22,7 @@ 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.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
 import org.apache.slider.funtest.framework.AgentCommandTestBase
@@ -42,7 +43,6 @@ public class AgentWebPagesIT extends AgentCommandTestBase
 
   static String APP_RESOURCE2 = "../slider-core/src/test/app_packages/test_command_log/resources_no_role.json"
 
-
   @Before
   public void prepareCluster() {
     setupCluster(CLUSTER)
@@ -56,6 +56,9 @@ public class AgentWebPagesIT extends AgentCommandTestBase
   @Test
   public void testAgentWeb() throws Throwable {
     describe("Create a 0-role cluster and make web queries against it")
+    
+    // verify the ws/ path is open for all HTTP verbs
+    assert SLIDER_CONFIG.getBoolean(SliderXmlConfKeys.X_DEV_INSECURE_WS, false)
     def clusterpath = buildClusterPath(CLUSTER)
     File launchReportFile = createTempJsonFile();
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,


[05/10] incubator-slider git commit: SLIDER-719 direct tests for PUT/POST/DELETE/HEAD all working with new client lib.

Posted by st...@apache.org.
SLIDER-719 direct tests for PUT/POST/DELETE/HEAD all working with new client lib.


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

Branch: refs/heads/develop
Commit: 95d4acb672b7a75e8b227e4d792178f83c0a543d
Parents: 372a5b7
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jan 8 15:45:03 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jan 8 15:45:03 2015 +0000

----------------------------------------------------------------------
 .../core/restclient/HttpOperationResponse.java  | 29 +++++++
 .../apache/slider/core/restclient/HttpVerb.java | 28 +++++--
 .../restclient/UrlConnectionOperations.java     | 65 +++++++++++++---
 .../rest/application/ApplicationResource.java   | 79 +++++++++++++++-----
 .../application/actions/RestActionPing.java     | 10 ++-
 .../application/resources/PingResource.java     |  5 ++
 .../standalone/TestStandaloneAgentWeb.groovy    | 51 +++++++++++--
 .../apache/slider/test/SliderTestUtils.groovy   | 25 ++++++-
 8 files changed, 243 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/95d4acb6/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
new file mode 100644
index 0000000..a5357a2
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java
@@ -0,0 +1,29 @@
+/*
+ * 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.core.restclient;
+
+/**
+ * A response for use as a return value from operations
+ */
+public class HttpOperationResponse {
+  
+  public int responseCode;
+  public String contentType;
+  public byte[] data;
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/95d4acb6/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
index 6767951..c040345 100644
--- a/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
@@ -18,26 +18,40 @@
 
 package org.apache.slider.core.restclient;
 
+/**
+ * Http verbs with details on what they support in terms of submit and
+ * response bodies.
+ * <p>
+ * Those verbs which do support bodies in the response MAY NOT return it;
+ * if the response code is 204 then the answer is "no body", but the operation
+ * is considered a success.
+ */
 public enum HttpVerb {
-  GET("GET", false),
-  POST("POST", true),
-  PUT("POST", true),
-  DELETE("DELETE", false),
-  HEAD("HEAD", false);
+  GET("GET", false, true),
+  POST("POST", true, true),
+  PUT("PUT", true, true),
+  DELETE("DELETE", false, true),
+  HEAD("HEAD", false, false);
   
   private final String verb;
   private final boolean hasUploadBody;
+  private final boolean hasResponseBody;
 
-  HttpVerb(String verb, boolean hasUploadBody) {
+  HttpVerb(String verb, boolean hasUploadBody, boolean hasResponseBody) {
     this.verb = verb;
     this.hasUploadBody = hasUploadBody;
+    this.hasResponseBody = hasResponseBody;
   }
 
   public String getVerb() {
     return verb;
   }
 
-  public boolean isHasUploadBody() {
+  public boolean hasUploadBody() {
     return hasUploadBody;
   }
+
+  public boolean hasResponseBody() {
+    return hasResponseBody;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/95d4acb6/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 eb5d4a7..4633414 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
@@ -30,8 +30,10 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.ws.rs.core.MediaType;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.HttpURLConnection;
 import java.net.URL;
 
@@ -45,10 +47,22 @@ public class UrlConnectionOperations extends Configured {
 
   private URLConnectionFactory connectionFactory;
 
+  private boolean useSpnego = false;
+
   public UrlConnectionOperations(Configuration conf) {
     super(conf);
     connectionFactory = URLConnectionFactory
-        .newDefaultURLConnectionFactory(conf);  }
+        .newDefaultURLConnectionFactory(conf);
+  }
+
+
+  public boolean isUseSpnego() {
+    return useSpnego;
+  }
+
+  public void setUseSpnego(boolean useSpnego) {
+    this.useSpnego = useSpnego;
+  }
 
   /**
    * Opens a url with read and connect timeouts
@@ -58,28 +72,59 @@ public class UrlConnectionOperations extends Configured {
    * @return URLConnection
    * @throws IOException
    */
-  public HttpURLConnection openConnection(URL url, boolean spnego) throws
+  public HttpURLConnection openConnection(URL url) throws
       IOException,
       AuthenticationException {
     Preconditions.checkArgument(url.getPort() != 0, "no port");
     HttpURLConnection conn =
-        (HttpURLConnection) connectionFactory.openConnection(url, spnego);
+        (HttpURLConnection) connectionFactory.openConnection(url, useSpnego);
     conn.setUseCaches(false);
     conn.setInstanceFollowRedirects(true);
     return conn;
   }
 
-  public byte[] execGet(URL url, boolean spnego) throws
+  public HttpOperationResponse execGet(URL url) throws
       IOException,
       AuthenticationException {
+    return execHttpOperation(HttpVerb.GET, url, null, "");
+  }
+
+  public HttpOperationResponse execHttpOperation(HttpVerb verb,
+      URL url,
+      byte[] payload,
+      String contentType)
+      throws IOException, AuthenticationException {
     HttpURLConnection conn = null;
+    HttpOperationResponse outcome = new HttpOperationResponse();
     int resultCode;
     byte[] body = null;
-    log.debug("GET {} spnego={}", url, spnego);
+    log.debug("{} {} spnego={}", verb, url, useSpnego);
 
+    boolean doOutput = verb.hasUploadBody();
+    if (doOutput) {
+      Preconditions.checkArgument(payload !=null,
+          "Null payload on a verb which expects one");
+    }
     try {
-      conn = openConnection(url, spnego);
+      conn = openConnection(url);
+      conn.setRequestMethod(verb.getVerb());
+      conn.setDoOutput(doOutput);
+      if (doOutput) {
+        conn.setRequestProperty("Content-Type", contentType);
+      }
+      
+
+      // now do the connection
+      conn.connect();
+      
+      if (doOutput) {
+        OutputStream output = conn.getOutputStream();
+        IOUtils.write(payload, output);
+        output.close();
+      }
+      
       resultCode = conn.getResponseCode();
+      outcome.contentType = conn.getContentType();
       InputStream stream = conn.getErrorStream();
       if (stream == null) {
         stream = conn.getInputStream();
@@ -93,11 +138,11 @@ public class UrlConnectionOperations extends Configured {
 
       }
     } catch (IOException e) {
-      throw NetUtils.wrapException(url.toString(), 
+      throw NetUtils.wrapException(url.toString(),
           url.getPort(), "localhost", 0, e);
 
     } catch (AuthenticationException e) {
-      throw new IOException("From " + url + ": " + e.toString(), e);
+      throw new IOException("From " + url + ": " + e, e);
 
     } finally {
       if (conn != null) {
@@ -105,7 +150,9 @@ public class UrlConnectionOperations extends Configured {
       }
     }
     uprateFaults(url.toString(), resultCode, body);
-    return body;
+    outcome.responseCode = resultCode;
+    outcome.data = body;
+    return outcome;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/95d4acb6/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 1bdf109..af310b1 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
@@ -46,12 +46,18 @@ import org.slf4j.LoggerFactory;
 
 import javax.inject.Singleton;
 import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
+import javax.ws.rs.HEAD;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
+
+import static javax.ws.rs.core.MediaType.*;
 import javax.ws.rs.core.UriInfo;
 import java.util.ArrayList;
 import java.util.List;
@@ -129,7 +135,7 @@ public class ApplicationResource extends AbstractSliderResource {
 
   @GET
   @Path("/")
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public List<String> getRoot() {
     return ROOT_ENTRIES;
   }
@@ -140,70 +146,70 @@ public class ApplicationResource extends AbstractSliderResource {
    */
   @GET
   @Path(MODEL)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public List<String> getModel() {
     return MODEL_ENTRIES;
   }
 
   @GET
   @Path(MODEL_DESIRED)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public AggregateConf getModelDesired() {
     return lookupAggregateConf(MODEL_DESIRED);
   }
   
   @GET
   @Path(MODEL_DESIRED_APPCONF)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public ConfTree getModelDesiredAppconf() {
     return lookupConfTree(MODEL_DESIRED_APPCONF);
   }
 
   @GET
   @Path(MODEL_DESIRED_RESOURCES)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public ConfTree getModelDesiredResources() {
     return lookupConfTree(MODEL_DESIRED_RESOURCES);
   }
   
   @GET
   @Path(MODEL_RESOLVED)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public AggregateConf getModelResolved() {
     return lookupAggregateConf(MODEL_RESOLVED);
   }
 
   @GET
   @Path(MODEL_RESOLVED_APPCONF)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public ConfTree getModelResolvedAppconf() {
     return lookupConfTree(MODEL_RESOLVED_APPCONF);
   }
 
   @GET
   @Path(MODEL_RESOLVED_RESOURCES)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public ConfTree getModelResolvedResources() {
     return lookupConfTree(MODEL_RESOLVED_RESOURCES);
   }
   
   @GET
   @Path(LIVE)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public List<String> getLive() {
     return LIVE_ENTRIES;
   }
 
   @GET
   @Path(LIVE_RESOURCES)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public Object getLiveResources() {
     return lookupConfTree(LIVE_RESOURCES);
   }
   
   @GET
   @Path(LIVE_CONTAINERS)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public Map<String, SerializedContainerInformation> getLiveContainers() {
     try {
       return (Map<String, SerializedContainerInformation>)cache.lookup(
@@ -215,7 +221,7 @@ public class ApplicationResource extends AbstractSliderResource {
 
   @GET
   @Path(LIVE_CONTAINERS + "/{containerId}")
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public SerializedContainerInformation getLiveContainer(
       @PathParam("containerId") String containerId) {
     try {
@@ -230,7 +236,7 @@ public class ApplicationResource extends AbstractSliderResource {
 
   @GET
   @Path(LIVE_COMPONENTS)
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public Map<String, SerializedComponentInformation> getLiveComponents() {
     try {
       return (Map<String, SerializedComponentInformation>) cache.lookup(
@@ -242,7 +248,7 @@ public class ApplicationResource extends AbstractSliderResource {
   
   @GET
   @Path(LIVE_COMPONENTS + "/{component}")
-  @Produces({MediaType.APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
   public SerializedComponentInformation getLiveComponent(
       @PathParam("component") String component) {
     try {
@@ -297,9 +303,46 @@ public class ApplicationResource extends AbstractSliderResource {
 
   @GET
   @Path(ACTION_PING)
-  @Produces({MediaType.APPLICATION_JSON})
-  public Object actionPing(@Context HttpServletRequest request,
+  @Produces({APPLICATION_JSON})
+  public Object actionPingGet(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    return new RestActionPing().ping(request, uriInfo, "");
+  }
+  
+  @POST
+  @Path(ACTION_PING)
+  @Produces({APPLICATION_JSON})
+  public Object actionPingPost(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo,
+      String body) {
+    return new RestActionPing().ping(request, uriInfo, body);
+  }
+  
+  @PUT
+  @Path(ACTION_PING)
+  @Consumes({TEXT_PLAIN})
+  @Produces({APPLICATION_JSON})
+  public Object actionPingPut(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo,
+      String body) {
+    return new RestActionPing().ping(request, uriInfo, body);
+  }
+  
+  @DELETE
+  @Path(ACTION_PING)
+  @Consumes({APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
+  public Object actionPingDelete(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    return new RestActionPing().ping(request, uriInfo, "");
+  }
+  
+  @HEAD
+  @Path(ACTION_PING)
+  @Produces({APPLICATION_JSON})
+  public Object actionPingHead(@Context HttpServletRequest request,
       @Context UriInfo uriInfo) {
-    return new RestActionPing().ping(request, uriInfo);
+    return new RestActionPing().ping(request, uriInfo, "");
   }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/95d4acb6/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
index 6113e1e..65126ac 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
@@ -34,15 +34,17 @@ public class RestActionPing {
   public RestActionPing() {
   }
   
-  public Object ping(@Context HttpServletRequest request,
-      @Context UriInfo uriInfo) {
-    log.info("Ping {}", request.getMethod());
+  public Object ping(HttpServletRequest request, UriInfo uriInfo, String body) {
+    String verb = request.getMethod();
+    log.info("Ping {}", verb);
     PingResource pingResource = new PingResource();
     pingResource.time = System.currentTimeMillis();
+    pingResource.verb = verb;
+    pingResource.body = body;
     String text = 
         String.format(Locale.ENGLISH,
             "Ping verb %s received at %tc",
-            request.getMethod(), pingResource.time);
+            verb, pingResource.time);
     pingResource.text = text;
     return pingResource;
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/95d4acb6/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java
index 7e5396c..3f67c55 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java
@@ -26,13 +26,18 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
 public class PingResource {
   public long time;
   public String text;
+  public String verb;
+  public String body;
 
   @Override
   public String toString() {
+    
     final StringBuilder sb =
         new StringBuilder("PingResource{");
     sb.append("time=").append(time);
+    sb.append(", verb=").append(verb);
     sb.append(", text='").append(text).append('\'');
+    sb.append(", body='").append(body).append('\'');
     sb.append('}');
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/95d4acb6/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
index 1018a02..d5be646 100644
--- 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
@@ -29,9 +29,13 @@ 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
@@ -51,6 +55,7 @@ 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 {
@@ -187,9 +192,11 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
     assert amFullInfo.containers[0] == amContainerId
 
     testRESTModel(appmaster)
-    testPing(appmaster)
-    
     
+    // PUT & POST &c must go direct for now
+    String wsroot = appendToURL(realappmaster, SLIDER_CONTEXT_ROOT)
+    testPing(realappmaster)
+
   }
 
   public void testRESTModel(String appmaster) {
@@ -200,14 +207,14 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
         ApplicationResource.MODEL_ENTRIES)
 
     def unresolvedConf = fetchType(AggregateConf, appmaster, MODEL_DESIRED)
-    log.info "Unresolved \n$unresolvedConf"
+//    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"
+//    log.info "Resolved \n$resolvedConf"
     assert resolvedConf.appConfOperations.getComponentOpt(
         sam, TEST_GLOBAL_OPTION, "") == TEST_GLOBAL_OPTION_PRESENT
 
@@ -239,11 +246,41 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
   }
 
   public void testPing(String appmaster) {
-    describe "ping"
-    def pinged = fetchType(PingResource, appmaster, ACTION_PING)
-    log.info "Ping: $pinged"
+    // 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/95d4acb6/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 b496d45..87e3206 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
@@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileStatus
 import org.apache.hadoop.fs.FileSystem as HadoopFS
 import org.apache.hadoop.fs.Path
-import org.apache.hadoop.hdfs.web.URLConnectionFactory
 import org.apache.hadoop.net.NetUtils
 import org.apache.hadoop.service.ServiceStateException
 import org.apache.hadoop.util.Shell
@@ -596,8 +595,8 @@ class SliderTestUtils extends Assert {
 
     log.info("Fetching HTTP content at " + path);
     URL url = new URL(path)
-    def bytes = connectionFactory.execGet(url, false)
-    String body = new String(bytes)
+    def outcome = connectionFactory.execGet(url)
+    String body = new String(outcome.data)
     return body;
   }
 
@@ -1144,16 +1143,34 @@ class SliderTestUtils extends Assert {
     }
   }
 
+  /**
+   * Get a web page and deserialize the supplied JSON into
+   * an instance of the specific class.
+   * @param clazz class to deserialize to
+   * @param appmaster URL to base AM
+   * @param subpath subpath under AM
+   * @return the parsed data type
+   */
   public <T> T fetchType(
       Class<T> clazz, String appmaster, String subpath) {
-    JsonSerDeser serDeser = new JsonSerDeser(clazz)
 
     def json = getWebPage(
         appmaster,
         RestPaths.SLIDER_PATH_APPLICATION + subpath)
+    return (T) deser(clazz, json);
+  }
+
+  public <T> T deser(Class<T> clazz, String json) {
+    JsonSerDeser serDeser = new JsonSerDeser(clazz)
     T ctree = (T) serDeser.fromJson(json)
     return ctree
   }
+
+  public <T> T deser(Class<T> clazz, byte[] data) {
+    JsonSerDeser serDeser = new JsonSerDeser(clazz)
+    T ctree = (T) serDeser.fromBytes(data)
+    return ctree
+  }
   
   public ConfTreeOperations fetchConfigTree(
       YarnConfiguration conf, String appmaster, String subpath) {


[07/10] incubator-slider git commit: SLIDER-719 new client lib shows up TestAgentAMManagementWS brittle at startup; replaced sleep with spinning poll of agent URL

Posted by st...@apache.org.
SLIDER-719 new client lib shows up TestAgentAMManagementWS brittle at startup; replaced sleep with spinning poll of agent URL


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

Branch: refs/heads/develop
Commit: 63dc3bb63910233a97fd754d2caa3f7c0fa52748
Parents: f769661
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 9 13:53:02 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 9 13:53:02 2015 +0000

----------------------------------------------------------------------
 .../slider/providers/agent/TestAgentAMManagementWS.groovy    | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/63dc3bb6/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
index 887ca89..2e2d6bf 100644
--- a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
@@ -152,11 +152,9 @@ class TestAgentAMManagementWS extends AgentTestBase {
         log.info("stacks is ${liveURL}stacks")
         log.info("conf   is ${liveURL}conf")
 
-
-        def sleeptime = 10
-        log.info "sleeping for $sleeptime seconds"
-        Thread.sleep(sleeptime * 1000)
-
+        execHttpRequest(WEB_STARTUP_TIME) {
+          GET(agent_url)
+        }
 
         String page = fetchWebPageWithoutError(agent_url);
         log.info(page);


[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

Posted by st...@apache.org.
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)
   }
 
 }


[04/10] incubator-slider git commit: SLIDER-719 Create slider REST client library

Posted by st...@apache.org.
SLIDER-719 Create slider REST client library


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

Branch: refs/heads/develop
Commit: 372a5b79670059271ad4cab93b04f0ce3512eec2
Parents: 6e1453b
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jan 8 12:33:52 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jan 8 12:33:52 2015 +0000

----------------------------------------------------------------------
 .../apache/slider/core/restclient/HttpVerb.java |  43 ++++++
 .../restclient/UrlConnectionOperations.java     | 149 +++++++++++++++++++
 .../apache/slider/test/SliderTestUtils.groovy   |  51 ++-----
 3 files changed, 206 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/372a5b79/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
new file mode 100644
index 0000000..6767951
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
@@ -0,0 +1,43 @@
+/*
+ * 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.core.restclient;
+
+public enum HttpVerb {
+  GET("GET", false),
+  POST("POST", true),
+  PUT("POST", true),
+  DELETE("DELETE", false),
+  HEAD("HEAD", false);
+  
+  private final String verb;
+  private final boolean hasUploadBody;
+
+  HttpVerb(String verb, boolean hasUploadBody) {
+    this.verb = verb;
+    this.hasUploadBody = hasUploadBody;
+  }
+
+  public String getVerb() {
+    return verb;
+  }
+
+  public boolean isHasUploadBody() {
+    return hasUploadBody;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/372a5b79/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
new file mode 100644
index 0000000..eb5d4a7
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
@@ -0,0 +1,149 @@
+/*
+ * 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.core.restclient;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+/**
+ * Operations on the JDK UrlConnection class. This uses WebHDFS
+ * methods to set up the operations.
+ */
+public class UrlConnectionOperations extends Configured {
+  private static final Logger log =
+      LoggerFactory.getLogger(UrlConnectionOperations.class);
+
+  private URLConnectionFactory connectionFactory;
+
+  public UrlConnectionOperations(Configuration conf) {
+    super(conf);
+    connectionFactory = URLConnectionFactory
+        .newDefaultURLConnectionFactory(conf);  }
+
+  /**
+   * Opens a url with read and connect timeouts
+   *
+   * @param url
+   *          to open
+   * @return URLConnection
+   * @throws IOException
+   */
+  public HttpURLConnection openConnection(URL url, boolean spnego) throws
+      IOException,
+      AuthenticationException {
+    Preconditions.checkArgument(url.getPort() != 0, "no port");
+    HttpURLConnection conn =
+        (HttpURLConnection) connectionFactory.openConnection(url, spnego);
+    conn.setUseCaches(false);
+    conn.setInstanceFollowRedirects(true);
+    return conn;
+  }
+
+  public byte[] execGet(URL url, boolean spnego) throws
+      IOException,
+      AuthenticationException {
+    HttpURLConnection conn = null;
+    int resultCode;
+    byte[] body = null;
+    log.debug("GET {} spnego={}", url, spnego);
+
+    try {
+      conn = openConnection(url, spnego);
+      resultCode = conn.getResponseCode();
+      InputStream stream = conn.getErrorStream();
+      if (stream == null) {
+        stream = conn.getInputStream();
+      }
+      if (stream != null) {
+        // read into a buffer.
+        body = IOUtils.toByteArray(stream);
+      } else {
+        // no body: 
+        log.debug("No body in response");
+
+      }
+    } catch (IOException e) {
+      throw NetUtils.wrapException(url.toString(), 
+          url.getPort(), "localhost", 0, e);
+
+    } catch (AuthenticationException e) {
+      throw new IOException("From " + url + ": " + e.toString(), e);
+
+    } finally {
+      if (conn != null) {
+        conn.disconnect();
+      }
+    }
+    uprateFaults(url.toString(), resultCode, body);
+    return body;
+  }
+
+  /**
+   * Uprate error codes 400 and up into faults; 
+   * 404 is converted to a {@link NotFoundException},
+   * 401 to {@link ForbiddenException}
+   * @param url URL as string
+   * @param resultCode response from the request
+   * @param body optional body of the request
+   * @throws IOException if the result was considered a failure
+   */
+  public static void uprateFaults(String url,
+      int resultCode, byte[] body)
+      throws IOException {
+
+    if (resultCode < 400) {
+      //success
+      return;
+    }
+    if (resultCode == 404) {
+      throw new NotFoundException(url);
+    }
+    if (resultCode == 401) {
+      throw new ForbiddenException(url);
+    }
+    // all other error codes
+    String bodyAsString;
+    if (body != null && body.length > 0) {
+      bodyAsString = new String(body);
+    } else {
+      bodyAsString = "";
+    }
+    String message = "Request to " + url +
+                     " failed with exit code " + resultCode
+                     + ", body length " + bodyAsString.length()
+                     + ":\n" + bodyAsString;
+    log.error(message);
+    throw new IOException(message);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/372a5b79/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 cd78c6c..b496d45 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
@@ -56,6 +56,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.UrlConnectionOperations
 import org.apache.slider.server.appmaster.web.HttpCacheHeaders
 import org.apache.slider.server.appmaster.web.rest.RestPaths
 import org.apache.slider.server.services.workflow.ForkedProcessService
@@ -500,7 +501,7 @@ class SliderTestUtils extends Assert {
     try {
       resultCode = client.executeMethod(get);
     } catch (IOException e) {
-      throw NetUtils.wrapException(url, 0, null, 0, e)
+      throw NetUtils.wrapException(url, destURI.port, "localhost", 0, e)
     }
 
     def body = get.responseBodyAsString
@@ -518,15 +519,18 @@ class SliderTestUtils extends Assert {
   public static void uprateFaults(String url, int resultCode, String body) {
 
     if (resultCode == 404) {
-      throw new NotFoundException(url)
+      throw new NotFoundException(url);
     }
     if (resultCode == 401) {
-      throw new ForbiddenException(url)
+      throw new ForbiddenException(url);
     }
     if (!(resultCode >= 200 && resultCode < 400)) {
-      def message = "Request to $url failed with exit code $resultCode, body length ${body?.length()}:\n$body"
-      log.error(message)
-      throw new IOException(message)
+      String message = "Request to " + url +
+                       " failed with exit code " +
+                       resultCode + ", body length " +
+                       body?.length() + ":\n" + "body;"
+      log.error(message);
+      throw new IOException(message);
     }
   }
 
@@ -566,11 +570,10 @@ class SliderTestUtils extends Assert {
     throw ex;
   } 
 
-  static URLConnectionFactory connectionFactory
+  static UrlConnectionOperations connectionFactory
 
   public static def initConnectionFactory(Configuration conf) {
-    connectionFactory = URLConnectionFactory
-        .newDefaultURLConnectionFactory(conf);
+    connectionFactory = new UrlConnectionOperations(conf);
   }
 
 
@@ -593,34 +596,8 @@ class SliderTestUtils extends Assert {
 
     log.info("Fetching HTTP content at " + path);
     URL url = new URL(path)
-    assert url.port != 0
-    HttpURLConnection conn = null;
-    int resultCode = 0
-    def body = ""
-    try {
-      conn = (HttpURLConnection) connectionFactory.openConnection(url);
-      conn.instanceFollowRedirects = true;
-      conn.connect()
-      
-
-      resultCode = conn.responseCode
-      
-      if (connectionChecks) {
-        connectionChecks(conn)
-      }
-      
-      InputStream stream = conn.errorStream;
-      if (stream == null) {
-        stream = conn.inputStream;
-      }
-
-      body = stream ? stream.text : "(no body)"
-    } catch (IOException e) {
-      throw NetUtils.wrapException(url.toString(), 0, null, 0, e)
-    } finally {
-      conn?.disconnect()
-    }
-    uprateFaults(path, resultCode, body)
+    def bytes = connectionFactory.execGet(url, false)
+    String body = new String(bytes)
     return body;
   }
 


[03/10] incubator-slider git commit: SLIDER-710 writing a ping operation to test verb support

Posted by st...@apache.org.
SLIDER-710 writing a ping operation to test verb support


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

Branch: refs/heads/develop
Commit: 6e1453bbb4752a9bb2ab8bd66d250e34417dbea5
Parents: e5fb7f8
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 7 15:16:02 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 7 15:16:02 2015 +0000

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  | 13 +++---
 .../org/apache/slider/common/SliderKeys.java    | 12 +++--
 .../apache/slider/common/SliderXmlConfKeys.java |  2 -
 .../slider/common/tools/ConfigHelper.java       | 21 ++++++---
 .../apache/slider/common/tools/SliderUtils.java | 15 +++---
 .../server/appmaster/SliderAppMaster.java       | 25 ++++++----
 .../server/appmaster/web/rest/RestPaths.java    |  3 ++
 .../rest/application/ApplicationResource.java   | 17 +++++++
 .../application/actions/RestActionPing.java     | 49 ++++++++++++++++++++
 .../application/resources/PingResource.java     | 39 ++++++++++++++++
 .../main/resources/org/apache/slider/slider.xml |  6 +++
 .../standalone/TestStandaloneAgentWeb.groovy    |  9 ++++
 .../tools/TestClientResourceRegistration.groovy |  2 +-
 .../funtest/framework/FuntestProperties.groovy  |  2 +-
 14 files changed, 178 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index f7b558e..2d1af35 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -252,8 +252,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    Configuration clientConf = SliderUtils.loadClientConfigurationResource();
-    ConfigHelper.mergeConfigurations(conf, clientConf, CLIENT_RESOURCE, true);
+    Configuration clientConf = SliderUtils.loadSliderClientXML();
+    ConfigHelper.mergeConfigurations(conf, clientConf, SLIDER_CLIENT_XML, true);
     serviceArgs.applyDefinitions(conf);
     serviceArgs.applyFileSystemBinding(conf);
     // init security with our conf
@@ -1437,8 +1437,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     String libdir = "lib";
     Path libPath = new Path(tempPath, libdir);
     sliderFileSystem.getFileSystem().mkdirs(libPath);
-    log.debug("FS={}, tempPath={}, libdir={}", sliderFileSystem.toString(),
-              tempPath, libPath);
+    log.debug("FS={}, tempPath={}, libdir={}",
+        sliderFileSystem, tempPath, libPath);
+ 
     // set local resources for the application master
     // local files or archives as needed
     // In this scenario, the jar file for the application master is part of the local resources
@@ -1593,7 +1594,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     JavaCommandLineBuilder commandLine = new JavaCommandLineBuilder();
     // insert any JVM options);
     sliderAM.addJVMOptions(instanceDefinition, commandLine);
-    // enable asserts if the text option is set
+    // enable asserts
     commandLine.enableJavaAssertions();
     
     // if the conf dir has a log4j-server.properties, switch to that
@@ -3044,7 +3045,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   private void actionDiagnosticCredentials() throws BadConfigException,
       IOException {
     if (SliderUtils.isHadoopClusterSecure(SliderUtils
-        .loadClientConfigurationResource())) {
+        .loadSliderClientXML())) {
       String credentialCacheFileDescription = null;
       try {
         credentialCacheFileDescription = SliderUtils.checkCredentialCacheFile();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java b/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
index 701ef14..8437087 100644
--- a/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
@@ -112,7 +112,7 @@ public interface SliderKeys extends SliderXmlConfKeys {
    * {@value}
    */
   String SLIDER_XML = "org/apache/slider/slider.xml";
-
+  
   String CLUSTER_DIRECTORY = "cluster";
 
   String PACKAGE_DIRECTORY = "package";
@@ -158,13 +158,12 @@ public interface SliderKeys extends SliderXmlConfKeys {
    * name of the Slider client resource
    * loaded when the service is loaded.
    */
-  String CLIENT_RESOURCE = "slider-client.xml";
+  String SLIDER_CLIENT_XML = "slider-client.xml";
 
   /**
    * The name of the resource to put on the classpath
-   * This only goes up on a real cluster, not a test run.
    */
-  String SERVER_RESOURCE = "slider-server.xml";
+  String SLIDER_SERVER_XML = "slider-server.xml";
 
   String TMP_LOGDIR_PREFIX = "/tmp/slider-";
   String TMP_DIR_PREFIX = "tmp";
@@ -210,7 +209,12 @@ public interface SliderKeys extends SliderXmlConfKeys {
   String AM_FILTER_NAME =
       "org.apache.hadoop.yarn.server.webproxy.amfilter.AmFilterInitializer";
 
+  /**
+   * Allowed port range. This MUST be set in app_conf/global.
+   * {@value}
+   */
   String KEY_ALLOWED_PORT_RANGE = "site.global.slider.allowed.ports";
+  
   /**
    * Allowed port range
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java b/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
index 0672955..0d1121d 100644
--- a/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
@@ -18,9 +18,7 @@
 
 package org.apache.slider.common;
 
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.registry.client.api.RegistryConstants;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
  * These are the keys that can be added to <code>conf/slider-client.xml</code>.

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java b/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
index af5ceee..c7b79f0 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
@@ -489,8 +489,7 @@ public class ConfigHelper {
    * @return the URL or null
    */
   public static URL registerDefaultResource(String resource) {
-    URL resURL = ConfigHelper.class.getClassLoader()
-                                .getResource(resource);
+    URL resURL = getResourceUrl(resource);
     if (resURL != null) {
       Configuration.addDefaultResource(resource);
     }
@@ -505,8 +504,7 @@ public class ConfigHelper {
    */
   public static Configuration loadFromResource(String resource) {
     Configuration conf = new Configuration(false);
-    URL resURL = ConfigHelper.class.getClassLoader()
-                                .getResource(resource);
+    URL resURL = getResourceUrl(resource);
     if (resURL != null) {
       log.debug("loaded resources from {}", resURL);
       conf.addResource(resource);
@@ -518,7 +516,17 @@ public class ConfigHelper {
   }
 
   /**
-   * Load a resource that must be there
+   * Get the URL to a resource, null if not on the CP
+   * @param resource resource to look for
+   * @return the URL or null
+   */
+  public static URL getResourceUrl(String resource) {
+    return ConfigHelper.class.getClassLoader()
+                                  .getResource(resource);
+  }
+
+  /**
+   * Load a resource that must be on the classpath
    * @param resource the resource name
    * @return the loaded configuration
    * @throws FileNotFoundException if the resource is missing
@@ -526,8 +534,7 @@ public class ConfigHelper {
   public static Configuration loadMandatoryResource(String resource)
       throws FileNotFoundException {
     Configuration conf = new Configuration(false);
-    URL resURL = ConfigHelper.class.getClassLoader()
-                                .getResource(resource);
+    URL resURL = getResourceUrl(resource);
     if (resURL != null) {
       log.debug("loaded resources from {}", resURL);
       conf.addResource(resource);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index af034b1..aeffe6c 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -1304,23 +1304,22 @@ public final class SliderUtils {
 
   /**
    * Register the client resource in
-   * {@link SliderKeys#CLIENT_RESOURCE}
+   * {@link SliderKeys#SLIDER_CLIENT_XML}
    * for Configuration instances.
    *
    * @return true if the resource could be loaded
    */
   public static URL registerClientResource() {
-    return ConfigHelper.registerDefaultResource(SliderKeys.CLIENT_RESOURCE);
+    return ConfigHelper.registerDefaultResource(SliderKeys.SLIDER_CLIENT_XML);
   }
-
-
+  
   /**
    * Attempt to load the slider client resource. If the
    * resource is not on the CP an empty config is returned.
    * @return a config
    */
-  public static Configuration loadClientConfigurationResource() {
-    return ConfigHelper.loadFromResource(SliderKeys.CLIENT_RESOURCE);
+  public static Configuration loadSliderClientXML() {
+    return ConfigHelper.loadFromResource(SliderKeys.SLIDER_CLIENT_XML);
   }
 
   /**
@@ -2025,7 +2024,7 @@ public final class SliderUtils {
    */
   public static String getClientConfigPath() {
     URL path = ConfigHelper.class.getClassLoader().getResource(
-        SliderKeys.CLIENT_RESOURCE);
+        SliderKeys.SLIDER_CLIENT_XML);
     return path.toString();
   }
 
@@ -2037,7 +2036,7 @@ public final class SliderUtils {
    */
   public static void validateClientConfigFile() throws IOException {
     URL resURL = SliderVersionInfo.class.getClassLoader().getResource(
-        SliderKeys.CLIENT_RESOURCE);
+        SliderKeys.SLIDER_CLIENT_XML);
     if (resURL == null) {
       throw new IOException(
           "slider-client.xml doesn't exist on the path: "

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index c34c692..cc9bf82 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -405,11 +405,17 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   public synchronized void serviceInit(Configuration conf) throws Exception {
     // slider client if found
     
-    Configuration customConf = SliderUtils.loadClientConfigurationResource();
+    Configuration customConf = SliderUtils.loadSliderClientXML();
     // Load in the server configuration - if it is actually on the Classpath
-    Configuration serverConf =
-      ConfigHelper.loadFromResource(SERVER_RESOURCE);
-    ConfigHelper.mergeConfigurations(customConf, serverConf, SERVER_RESOURCE, true);
+    URL serverXmlUrl = ConfigHelper.getResourceUrl(SLIDER_SERVER_XML);
+    if (serverXmlUrl != null) {
+
+      log.info("Loading {} at {}", SLIDER_SERVER_XML, serverXmlUrl);
+      Configuration serverConf =
+          ConfigHelper.loadFromResource(SLIDER_SERVER_XML);
+      ConfigHelper.mergeConfigurations(customConf, serverConf,
+          SLIDER_SERVER_XML, true);
+    }
     serviceArgs.applyDefinitions(customConf);
     serviceArgs.applyFileSystemBinding(customConf);
     // conf now contains all customizations
@@ -420,14 +426,14 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     // sort out the location of the AM
     String rmAddress = createAction.getRmAddress();
     if (rmAddress != null) {
-      log.debug("Setting rm address from the command line: {}", rmAddress);
+      log.debug("Setting RM address from the command line: {}", rmAddress);
       SliderUtils.setRmSchedulerAddress(customConf, rmAddress);
     }
 
     log.info("AM configuration:\n{}",
         ConfigHelper.dumpConfigToString(customConf));
 
-    ConfigHelper.mergeConfigurations(conf, customConf, CLIENT_RESOURCE, true);
+    ConfigHelper.mergeConfigurations(conf, customConf, SLIDER_CLIENT_XML, true);
     //init security with our conf
     if (SliderUtils.isHadoopClusterSecure(conf)) {
       log.info("Secure mode with kerberos realm {}",
@@ -495,11 +501,14 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   @Override // RunService
   public Configuration bindArgs(Configuration config, String... args) throws
                                                                       Exception {
+    // let the superclass process it
+    Configuration superConf = super.bindArgs(config, args);
+    //yarn-ify
     YarnConfiguration yarnConfiguration = new YarnConfiguration(
-        super.bindArgs(config, args));
+        superConf);
     serviceArgs = new SliderAMArgs(args);
     serviceArgs.parse();
-    //yarn-ify
+
     return SliderUtils.patchConfiguration(yarnConfiguration);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
index f637190..58520f5 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
@@ -142,4 +142,7 @@ public class RestPaths {
   public static final String MODEL_RESOLVED_APPCONF = MODEL_RESOLVED +"/appconf";
   public static final String MODEL_RESOLVED_RESOURCES = MODEL_RESOLVED +"/resources";
   public static final String MODEL_INTERNAL = "/model/internal";
+
+  public static final String ACTION = "/action";
+  public static final String ACTION_PING = ACTION + "/ping";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/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 48540d4..1bdf109 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
@@ -40,15 +40,19 @@ import org.apache.slider.server.appmaster.web.rest.application.resources.LiveCon
 import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
 import org.apache.slider.server.appmaster.web.rest.application.resources.LiveComponentsRefresher;
 import org.apache.slider.server.appmaster.web.rest.application.resources.LiveResourcesRefresher;
+import org.apache.slider.server.appmaster.web.rest.application.actions.RestActionPing;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.inject.Singleton;
+import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.UriInfo;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -73,6 +77,12 @@ public class ApplicationResource extends AbstractSliderResource {
   public static final List<String> MODEL_ENTRIES =
       toJsonList("desired", "resolved");
 
+  /**
+   * This is the cache of all content ... each entry is
+   * designed to be self-refreshing on get operations, 
+   * so is never very out of date, yet many GETs don't
+   * overload the rest of the system.
+   */
   private final ContentCache cache = new ContentCache();
   private final StateAccessForProviders state;
 
@@ -285,4 +295,11 @@ public class ApplicationResource extends AbstractSliderResource {
     }
   }
 
+  @GET
+  @Path(ACTION_PING)
+  @Produces({MediaType.APPLICATION_JSON})
+  public Object actionPing(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    return new RestActionPing().ping(request, uriInfo);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
new file mode 100644
index 0000000..6113e1e
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
@@ -0,0 +1,49 @@
+/*
+ * 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.server.appmaster.web.rest.application.actions;
+
+import org.apache.slider.server.appmaster.web.rest.application.resources.PingResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.UriInfo;
+import java.util.Locale;
+
+public class RestActionPing {
+  private static final Logger log =
+      LoggerFactory.getLogger(RestActionPing.class);
+
+  public RestActionPing() {
+  }
+  
+  public Object ping(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    log.info("Ping {}", request.getMethod());
+    PingResource pingResource = new PingResource();
+    pingResource.time = System.currentTimeMillis();
+    String text = 
+        String.format(Locale.ENGLISH,
+            "Ping verb %s received at %tc",
+            request.getMethod(), pingResource.time);
+    pingResource.text = text;
+    return pingResource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java
new file mode 100644
index 0000000..7e5396c
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/PingResource.java
@@ -0,0 +1,39 @@
+/*
+ * 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.server.appmaster.web.rest.application.resources;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class PingResource {
+  public long time;
+  public String text;
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("PingResource{");
+    sb.append("time=").append(time);
+    sb.append(", text='").append(text).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/main/resources/org/apache/slider/slider.xml
----------------------------------------------------------------------
diff --git a/slider-core/src/main/resources/org/apache/slider/slider.xml b/slider-core/src/main/resources/org/apache/slider/slider.xml
index 37ac65c..262a764 100644
--- a/slider-core/src/main/resources/org/apache/slider/slider.xml
+++ b/slider-core/src/main/resources/org/apache/slider/slider.xml
@@ -27,4 +27,10 @@
     <name>slider.provider.agent</name>
     <value>org.apache.slider.providers.agent.AgentProviderFactory</value>
   </property>
+  <property>
+    <name>slider.dev.ws.insecure</name>
+    <description>switch to insecure WS path</description>
+    <value>true</value>
+  </property>
+  
 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/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
index 7a62c83..1018a02 100644
--- 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
@@ -30,6 +30,7 @@ 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.server.appmaster.web.rest.application.ApplicationResource
+import org.apache.slider.server.appmaster.web.rest.application.resources.PingResource
 
 import static org.apache.slider.api.ResourceKeys.*
 import static org.apache.slider.api.StatusKeys.*
@@ -186,6 +187,7 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
     assert amFullInfo.containers[0] == amContainerId
 
     testRESTModel(appmaster)
+    testPing(appmaster)
     
     
   }
@@ -236,5 +238,12 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
     assert entries.containsAll(list)
   }
 
+  public void testPing(String appmaster) {
+    describe "ping"
+    def pinged = fetchType(PingResource, appmaster, ACTION_PING)
+    log.info "Ping: $pinged"
+    
+
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-core/src/test/groovy/org/apache/slider/common/tools/TestClientResourceRegistration.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/common/tools/TestClientResourceRegistration.groovy b/slider-core/src/test/groovy/org/apache/slider/common/tools/TestClientResourceRegistration.groovy
index b217329..81fffaa 100644
--- a/slider-core/src/test/groovy/org/apache/slider/common/tools/TestClientResourceRegistration.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/common/tools/TestClientResourceRegistration.groovy
@@ -87,7 +87,7 @@ class TestClientResourceRegistration {
    */
   @Test
   public void testLoadRes() throws Throwable {
-    Configuration conf = SliderUtils.loadClientConfigurationResource()
+    Configuration conf = SliderUtils.loadSliderClientXML()
     assert conf.get(KEY_RESOURCE_ORIGIN) == "test/resources"
     String hostname = "nosuchhost:0"
     conf.set(YarnConfiguration.RM_ADDRESS, hostname)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e1453bb/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FuntestProperties.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FuntestProperties.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FuntestProperties.groovy
index 2b13d13..3bc9263 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FuntestProperties.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/FuntestProperties.groovy
@@ -48,7 +48,7 @@ public interface FuntestProperties extends SliderXMLConfKeysForTesting {
   String KEY_AM_RESTART_SLEEP_TIME = "slider.test.am.restart.time"
   int DEFAULT_AM_RESTART_SLEEP_TIME = 30000
 
-  String CLIENT_CONFIG_FILENAME = SliderKeys.CLIENT_RESOURCE
+  String CLIENT_CONFIG_FILENAME = SliderKeys.SLIDER_CLIENT_XML
   
   String ENV_SLIDER_CONF_DIR = "SLIDER_CONF_DIR"
   String ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR"


[09/10] incubator-slider git commit: SLIDER-710: make sure that when the back door is closed, the tests will switch to the proxy

Posted by st...@apache.org.
SLIDER-710: make sure that when the back door is closed, the tests will switch to the proxy


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

Branch: refs/heads/develop
Commit: 3ba98bff98d1076c09a6ac01c4d3f1d1899c9578
Parents: d2e8f88
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 9 15:31:12 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 9 15:33:55 2015 +0000

----------------------------------------------------------------------
 .../slider/agent/rest/RestTestDelegates.groovy  |  8 ++++----
 .../slider/agent/rest/TestStandaloneREST.groovy | 16 ++++++++++++---
 .../funtest/lifecycle/AgentWebPagesIT.groovy    | 21 ++++++++++++++++----
 3 files changed, 34 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3ba98bff/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 f5264b4..430ddf1 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
@@ -150,7 +150,7 @@ class RestTestDelegates extends SliderTestUtils {
    * with the global option
    * @param appmaster
    */
-  public void testRESTModel(String appmaster) {
+  public void testRESTModel() {
     describe "model"
 
     assertPathServesList(appmaster,
@@ -182,7 +182,7 @@ class RestTestDelegates extends SliderTestUtils {
     TEST_GLOBAL_OPTION_PRESENT
   }
 
-  public void testPing(String appmaster) {
+  public void testPing() {
     // GET
     String ping = appendToURL(appmaster, SLIDER_PATH_APPLICATION, ACTION_PING)
     describe "ping to AM URL $appmaster, ping URL $ping"
@@ -203,14 +203,14 @@ class RestTestDelegates extends SliderTestUtils {
   }
 
 
-  public HttpOperationResponse pingAction(
+  private HttpOperationResponse pingAction(
       HttpVerb verb,
       URL pingUrl,
       String payload) {
     return pingAction(connectionFactory, verb, pingUrl, payload)
   }
 
-  public HttpOperationResponse pingAction(
+  private HttpOperationResponse pingAction(
       UrlConnectionOperations ops, HttpVerb verb, URL pingUrl, String payload) {
     def pinged
     def outcome = ops.execHttpOperation(

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3ba98bff/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 582ddc7..4b907e1 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
@@ -22,6 +22,7 @@ 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.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource
 import org.apache.slider.client.SliderClient
@@ -83,6 +84,10 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     log.info GET(appmaster, SYSTEM_HEALTHCHECK)
     log.info GET(appmaster, SYSTEM_METRICS_JSON)
 
+    def wsBackDoorRequired = conf.getBoolean(
+        SliderXmlConfKeys.X_DEV_INSECURE_WS,
+        true)
+    
     RestTestDelegates proxied = new RestTestDelegates(appmaster)
     RestTestDelegates direct = new RestTestDelegates(realappmaster)
     
@@ -97,11 +102,16 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     describe "Application REST ${LIVE_RESOURCES}"
     proxied.testLiveResources()
 
-    proxied.testRESTModel(appmaster)
+    proxied.testRESTModel()
     
     // PUT & POST &c must go direct for now
-    direct.testPing(realappmaster)
-
+    direct.testPing()
+    // PUT & POST &c direct
+    direct.testPing()
+    if (!wsBackDoorRequired) {
+      // and via the proxy
+      proxied.testRESTModel()
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3ba98bff/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 c71df8a..9e5e2cd 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
@@ -25,6 +25,7 @@ import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
+import org.apache.slider.common.tools.ConfigHelper
 import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
 import org.apache.slider.funtest.framework.SliderShell
@@ -58,7 +59,15 @@ public class AgentWebPagesIT extends AgentCommandTestBase
     describe("Create a 0-role cluster and make web queries against it")
     
     // verify the ws/ path is open for all HTTP verbs
-    assert SLIDER_CONFIG.getBoolean(SliderXmlConfKeys.X_DEV_INSECURE_WS, false)
+    def sliderConfiguration = ConfigHelper.loadSliderConfiguration();
+
+    def wsBackDoorRequired = SLIDER_CONFIG.getBoolean(
+        SliderXmlConfKeys.X_DEV_INSECURE_WS,
+        true)
+    assert wsBackDoorRequired ==
+        sliderConfiguration.getBoolean(
+            SliderXmlConfKeys.X_DEV_INSECURE_WS,
+            false)
     def clusterpath = buildClusterPath(CLUSTER)
     File launchReportFile = createTempJsonFile();
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,
@@ -104,10 +113,14 @@ public class AgentWebPagesIT extends AgentCommandTestBase
     direct.testCodahaleOperations()
     proxied.testLiveResources()
 
-    proxied.testRESTModel(appmaster)
+    proxied.testRESTModel()
 
-    // PUT & POST &c must go direct for now
-    direct.testPing(realappmaster)
+    // PUT & POST &c direct
+    direct.testPing()
+    if (!wsBackDoorRequired) {
+      // and via the proxy
+      proxied.testRESTModel()
+    }
   }
 
 }


[10/10] incubator-slider git commit: Merge branch 'feature/SLIDER-710_Restore_insecure_WS_back_door_for_REST_dev' into develop

Posted by st...@apache.org.
Merge branch 'feature/SLIDER-710_Restore_insecure_WS_back_door_for_REST_dev' into develop


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

Branch: refs/heads/develop
Commit: aaeac3196e6659726cd000391db730d919334d46
Parents: 7a1a590 3ba98bf
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 9 15:34:10 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 9 15:34:10 2015 +0000

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  |  15 +-
 .../org/apache/slider/common/SliderKeys.java    |  12 +-
 .../common/SliderXMLConfKeysForTesting.java     |   9 +-
 .../apache/slider/common/SliderXmlConfKeys.java |   7 +-
 .../slider/common/tools/ConfigHelper.java       |  61 ++++-
 .../apache/slider/common/tools/SliderUtils.java |  15 +-
 .../launch/SerializedApplicationReport.java     |   2 +-
 .../core/restclient/HttpOperationResponse.java  |  29 +++
 .../apache/slider/core/restclient/HttpVerb.java |  57 +++++
 .../restclient/UrlConnectionOperations.java     | 198 +++++++++++++++
 .../slider/providers/SliderProviderFactory.java |   5 +
 .../server/appmaster/SliderAppMaster.java       | 142 +++++++----
 .../appmaster/web/rest/InsecureAmFilter.java    | 105 ++++++++
 .../web/rest/InsecureAmFilterInitializer.java   | 102 ++++++++
 .../server/appmaster/web/rest/RestPaths.java    |   3 +
 .../rest/application/ApplicationResource.java   |  90 +++++--
 .../application/actions/RestActionPing.java     |  51 ++++
 .../application/resources/PingResource.java     |  44 ++++
 .../main/resources/org/apache/slider/slider.xml |   6 +
 .../slider/agent/rest/RestTestDelegates.groovy  | 235 ++++++++++++++++++
 .../slider/agent/rest/TestStandaloneREST.groovy | 119 +++++++++
 .../standalone/TestStandaloneAgentWeb.groovy    | 240 -------------------
 .../tools/TestClientResourceRegistration.groovy |   2 +-
 .../agent/TestAgentAMManagementWS.groovy        |   8 +-
 .../apache/slider/test/SliderTestBase.groovy    |   4 +-
 .../apache/slider/test/SliderTestUtils.groovy   | 105 ++++----
 .../funtest/framework/CommandTestBase.groovy    |   1 +
 .../funtest/framework/FuntestProperties.groovy  |   2 +-
 .../funtest/lifecycle/AgentWebPagesIT.groovy    |  58 ++++-
 29 files changed, 1320 insertions(+), 407 deletions(-)
----------------------------------------------------------------------



[02/10] incubator-slider git commit: SLIDER-710 restore AM filter (with an XML conf key to enable it)

Posted by st...@apache.org.
SLIDER-710 restore AM filter (with an XML conf key to enable it)


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

Branch: refs/heads/develop
Commit: e5fb7f83ea1e7d45020aaa962bfe2518672450db
Parents: ceb21e1
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jan 7 14:19:56 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jan 7 14:19:56 2015 +0000

----------------------------------------------------------------------
 .../common/SliderXMLConfKeysForTesting.java     |   9 +-
 .../apache/slider/common/SliderXmlConfKeys.java |   5 +
 .../server/appmaster/SliderAppMaster.java       | 104 ++++++++++++-------
 .../appmaster/web/rest/InsecureAmFilter.java    | 101 ++++++++++++++++++
 .../web/rest/InsecureAmFilterInitializer.java   | 103 ++++++++++++++++++
 5 files changed, 278 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e5fb7f83/slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java b/slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java
index 41c61d4..0a7f292 100644
--- a/slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java
+++ b/slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java
@@ -22,14 +22,7 @@ package org.apache.slider.common;
  * Keys shared across tests
  */
 public interface SliderXMLConfKeysForTesting {
-
-  String KEY_TEST_HBASE_HOME = "slider.test.hbase.home";
-  String KEY_TEST_HBASE_TAR = "slider.test.hbase.tar";
-  String KEY_TEST_HBASE_APPCONF = "slider.test.hbase.appconf";
-  String KEY_TEST_ACCUMULO_HOME = "slider.test.accumulo.home";
-  String KEY_TEST_ACCUMULO_TAR = "slider.test.accumulo.tar";
-  String KEY_TEST_ACCUMULO_APPCONF = "slider.test.accumulo.appconf";
-
+  
   String KEY_TEST_THAW_WAIT_TIME = "slider.test.thaw.wait.seconds";
 
   int DEFAULT_THAW_WAIT_TIME_SECONDS = 60;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e5fb7f83/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java b/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
index 9b1316e..0672955 100644
--- a/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
@@ -143,4 +143,9 @@ public interface SliderXmlConfKeys {
    * The path to the python executable utilized to launch the agent.
    */
   String PYTHON_EXECUTABLE_PATH = "agent.python.exec.path";
+
+  /**
+   * Flag to enable the insecure AM filter: {@value}
+   */
+  String X_DEV_INSECURE_WS = "slider.dev.ws.insecure";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e5fb7f83/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 7b9f6db..c34c692 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -146,6 +146,7 @@ import org.apache.slider.server.appmaster.state.RoleInstance;
 import org.apache.slider.server.appmaster.state.RoleStatus;
 import org.apache.slider.server.appmaster.state.SimpleReleaseSelector;
 import org.apache.slider.server.appmaster.web.AgentService;
+import org.apache.slider.server.appmaster.web.rest.InsecureAmFilterInitializer;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentWebApp;
 import org.apache.slider.server.appmaster.web.SliderAMWebApp;
 import org.apache.slider.server.appmaster.web.WebAppApi;
@@ -385,6 +386,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    * The port for the web application
    */
   private int webAppPort;
+  private boolean securityEnabled;
 
   /**
    * Service Constructor
@@ -593,7 +595,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     securityConfiguration = new SecurityConfiguration(
         serviceConf, instanceDefinition, clustername);
     // obtain security state
-    boolean securityEnabled = securityConfiguration.isSecurityEnabled();
+    securityEnabled = securityConfiguration.isSecurityEnabled();
     // set the global security flag for the instance definition
     instanceDefinition.getAppConfOperations().set(
         KEY_SECURITY_ENABLED, securityEnabled);
@@ -607,9 +609,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       File parentFile = confDir.getParentFile();
       log.info("Parent dir {}:\n{}", parentFile, SliderUtils.listDir(parentFile));
     }
-
-    // IP filtering
-    serviceConf.set(HADOOP_HTTP_FILTER_INITIALIZERS, AM_FILTER_NAME);
     
     //get our provider
     MapOperations globalInternalOptions = getGlobalInternalOptions();
@@ -655,6 +654,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
     Map<String, String> envVars;
     List<Container> liveContainers;
+    
     /**
      * It is critical this section is synchronized, to stop async AM events
      * arriving while registering a restarting AM.
@@ -719,7 +719,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
         uploadServerCertForLocalization(clustername, fs);
       }
 
-      startAgentWebApp(appInformation, serviceConf);
 
       webAppPort = getPortToRequest();
       if (webAppPort == 0) {
@@ -795,7 +794,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
       providerService.validateApplicationConfiguration(instanceDefinition, 
                                                        confDir,
-                                                       securityEnabled);
+          securityEnabled);
 
       //determine the location for the role history data
       Path historyDir = new Path(clusterDirPath, HISTORY_DIR_NAME);
@@ -815,11 +814,11 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
           instanceDefinition.getName(), appState.getRolePriorityMap());
 
       // add the AM to the list of nodes in the cluster
-      
+
       appState.buildAppMasterNode(appMasterContainerID,
-                                  appMasterHostname,
+          appMasterHostname,
           webAppPort,
-                                  appMasterHostname + ":" + webAppPort);
+          appMasterHostname + ":" + webAppPort);
 
       // build up environment variables that the AM wants set in every container
       // irrespective of provider and role.
@@ -872,13 +871,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     service_user_name = RegistryUtils.currentUser();
     log.info("Registry service username ={}", service_user_name);
 
-    // now do the registration
-    registerServiceInstance(clustername, appid);
 
-    // log the YARN and web UIs
-    log.info("RM Webapp address {}", serviceConf.get(YarnConfiguration.RM_WEBAPP_ADDRESS));
-    log.info("slider Webapp address {}", appMasterTrackingUrl);
-    
     // declare the cluster initialized
     log.info("Application Master Initialization Completed");
     initCompleted.set(true);
@@ -891,8 +884,28 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
       startQueueProcessing();
 
-      deployWebApplication(serviceConf, webAppPort);
+      // Web service endpoints: initialize
+
+      WebAppApiImpl webAppApi =
+          new WebAppApiImpl(this,
+              stateForProviders,
+              providerService,
+              certificateManager,
+              registryOperations,
+              metricsAndMonitoring);
+      initAMFilterOptions(serviceConf);
 
+      // start the agent web app
+      startAgentWebApp(appInformation, serviceConf, webAppApi);
+      deployWebApplication(serviceConf, webAppPort, webAppApi);
+
+      // YARN Registry do the registration
+      registerServiceInstance(clustername, appid);
+
+      // log the YARN and web UIs
+      log.info("RM Webapp address {}",
+          serviceConf.get(YarnConfiguration.RM_WEBAPP_ADDRESS));
+      log.info("slider Webapp address {}", appMasterTrackingUrl);
 
       // Start the Slider AM provider
       sliderAMProvider.start();
@@ -917,17 +930,13 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    *   Creates and starts the web application, and adds a
    *   <code>WebAppService</code> service under the AM, to ensure
    *   a managed web application shutdown.
-   * 
-   * @param serviceConf AM configuration
+   *  @param serviceConf AM configuration
    * @param port port to deploy the web application on
+   * @param webAppApi web app API instance
    */
-  private void deployWebApplication(Configuration serviceConf, int port) {
-    WebAppApi webAppApi = new WebAppApiImpl(this,
-        stateForProviders,
-        providerService,
-        certificateManager,
-        registryOperations,
-        metricsAndMonitoring);
+  private void deployWebApplication(Configuration serviceConf,
+      int port, WebAppApiImpl webAppApi) {
+
     webApp = new SliderAMWebApp(webAppApi);
     WebApps.$for(SliderAMWebApp.BASE_PATH,
         WebAppApi.class,
@@ -1061,26 +1070,31 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     }
   }
 
+  /**
+   * Set up and start the agent web application 
+   * @param appInformation application information
+   * @param serviceConf service configuration
+   * @param webAppApi web app API instance to bind to
+   * @throws IOException
+   */
   private void startAgentWebApp(MapOperations appInformation,
-                                Configuration serviceConf) throws IOException {
+      Configuration serviceConf, WebAppApiImpl webAppApi) throws IOException {
     URL[] urls = ((URLClassLoader) AgentWebApp.class.getClassLoader() ).getURLs();
     StringBuilder sb = new StringBuilder("AM classpath:");
     for (URL url : urls) {
       sb.append("\n").append(url.toString());
     }
-    LOG_YARN.info(sb.append("\n").toString());
+    LOG_YARN.debug(sb.append("\n").toString());
+    initAMFilterOptions(serviceConf);
+
+
     // Start up the agent web app and track the URL for it
+    MapOperations appMasterConfig = getInstanceDefinition()
+        .getAppConfOperations().getComponent(SliderKeys.COMPONENT_AM);
     AgentWebApp agentWebApp = AgentWebApp.$for(AgentWebApp.BASE_PATH,
-                     new WebAppApiImpl(this,
-                         stateForProviders,
-                         providerService,
-                         certificateManager,
-                         registryOperations,
-                         metricsAndMonitoring),
+        webAppApi,
                      RestPaths.AGENT_WS_CONTEXT)
-        .withComponentConfig(getInstanceDefinition().getAppConfOperations()
-                                                    .getComponent(
-                                                        SliderKeys.COMPONENT_AM))
+        .withComponentConfig(appMasterConfig)
         .start();
     agentOpsUrl =
         "https://" + appMasterHostname + ":" + agentWebApp.getSecuredPort();
@@ -1102,6 +1116,24 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   }
 
   /**
+   * Set up the AM filter 
+   * @param serviceConf configuration to patch
+   */
+  private void initAMFilterOptions(Configuration serviceConf) {
+    // IP filtering
+    String amFilterName = AM_FILTER_NAME;
+
+    // This is here until YARN supports proxy & redirect operations
+    // on verbs other than GET, and is only supported for testing
+    if (serviceConf.getBoolean(SliderXmlConfKeys.X_DEV_INSECURE_WS, false)) {
+      log.warn("Insecure filter enabled: REST operations are unauthenticated");
+      amFilterName = InsecureAmFilterInitializer.NAME;
+    }
+
+    serviceConf.set(HADOOP_HTTP_FILTER_INITIALIZERS, amFilterName);
+  }
+
+  /**
    * This registers the service instance and its external values
    * @param instanceName name of this instance
    * @param appid application ID

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e5fb7f83/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
new file mode 100644
index 0000000..07b19e7
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
@@ -0,0 +1,101 @@
+/*
+ * 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.server.appmaster.web.rest;
+
+import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServlet;
+import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter;
+import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpPrincipal;
+import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpServletRequestWrapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.Cookie;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+
+/**
+ * This is a filter which is used to forward insecure operations
+ * There's some metrics to track all operations too
+ */
+public class InsecureAmFilter extends AmIpFilter {
+  public static final String WS_CONTEXT_ROOT = "slider.rest.context.root";
+  protected static final Logger log =
+      LoggerFactory.getLogger(InsecureAmFilter.class);
+
+  private String wsContextRoot;
+
+
+  @Override
+  public void init(FilterConfig conf) throws ServletException {
+    super.init(conf);
+    wsContextRoot = conf.getInitParameter(WS_CONTEXT_ROOT);
+  }
+
+  private void rejectNonHttpRequests(ServletRequest req) throws
+      ServletException {
+    if (!(req instanceof HttpServletRequest)) {
+      throw new ServletException("This filter only works for HTTP/HTTPS");
+    }
+  }  
+
+  @Override
+  public void doFilter(ServletRequest req,
+      ServletResponse resp,
+      FilterChain chain) throws IOException, ServletException {
+    rejectNonHttpRequests(req);
+    HttpServletRequest httpReq = (HttpServletRequest) req;
+    HttpServletResponse httpResp = (HttpServletResponse) resp;
+
+
+    if (!httpReq.getRequestURI().startsWith(wsContextRoot)) {
+      // hand off to the AM filter if it is not the context root
+      super.doFilter(req, resp, chain);
+      return;
+    }
+
+    String user = null;
+
+    if (httpReq.getCookies() != null) {
+      for (Cookie c : httpReq.getCookies()) {
+        if (WebAppProxyServlet.PROXY_USER_COOKIE_NAME.equals(c.getName())) {
+          user = c.getValue();
+          break;
+        }
+      }
+    }
+    
+    if (user == null) {
+      log.warn("Could not find " + WebAppProxyServlet.PROXY_USER_COOKIE_NAME
+               + " cookie, so user will not be set");
+      chain.doFilter(req, resp);
+    } else {
+      final AmIpPrincipal principal = new AmIpPrincipal(user);
+      ServletRequest requestWrapper = new AmIpServletRequestWrapper(httpReq,
+          principal);
+      chain.doFilter(requestWrapper, resp);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e5fb7f83/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
new file mode 100644
index 0000000..111d715
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
@@ -0,0 +1,103 @@
+/**
+ * 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.server.appmaster.web.rest;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.http.FilterInitializer;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class InsecureAmFilterInitializer extends FilterInitializer {
+  private static final String FILTER_NAME = "AM_PROXY_FILTER";
+  private static final String FILTER_CLASS =
+      InsecureAmFilter.class.getCanonicalName();
+  private static final String HTTPS_PREFIX = "https://";
+  private static final String HTTP_PREFIX = "http://";
+
+  static final String PROXY_HOSTS = "PROXY_HOSTS";
+  static final String PROXY_HOSTS_DELIMITER = ",";
+  static final String PROXY_URI_BASES = "PROXY_URI_BASES";
+  static final String PROXY_URI_BASES_DELIMITER = ",";
+
+  private Configuration configuration;
+
+  public static final String NAME =
+      "org.apache.slider.server.appmaster.web.InsecureAmFilterInitializer";
+
+  @Override
+  public void initFilter(FilterContainer container, Configuration conf) {
+    configuration = conf;
+    Map<String, String> params = new HashMap<String, String>();
+    String proxy = WebAppUtils.getProxyHostAndPort(conf);
+    String[] parts = proxy.split(":");
+    params.put(InsecureAmFilter.PROXY_HOST, parts[0]);
+    // todo:  eventually call WebAppUtils.getHttpSchemePrefix
+    params.put(InsecureAmFilter.PROXY_URI_BASE, getHttpSchemePrefix()
+                                                + proxy +
+                                                getApplicationWebProxyBase());
+    params.put(InsecureAmFilter.WS_CONTEXT_ROOT,
+        conf.get(InsecureAmFilter.WS_CONTEXT_ROOT));
+    container.addFilter(FILTER_NAME, FILTER_CLASS, params);
+  }
+
+  private void classicAmFilterInitializerInit(FilterContainer container,
+      Configuration conf) {
+    Map<String, String> params = new HashMap<String, String>();
+    List<String> proxies = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
+    StringBuilder sb = new StringBuilder();
+    for (String proxy : proxies) {
+      sb.append(proxy.split(":")[0]).append(PROXY_HOSTS_DELIMITER);
+    }
+    sb.setLength(sb.length() - 1);
+    params.put(PROXY_HOSTS, sb.toString());
+
+    String prefix = WebAppUtils.getHttpSchemePrefix(conf);
+    String proxyBase = getApplicationWebProxyBase();
+    sb = new StringBuilder();
+    for (String proxy : proxies) {
+      sb.append(prefix).append(proxy).append(proxyBase)
+        .append(PROXY_HOSTS_DELIMITER);
+    }
+    sb.setLength(sb.length() - 1);
+    params.put(PROXY_URI_BASES, sb.toString());
+
+  }
+
+  @VisibleForTesting
+  protected String getApplicationWebProxyBase() {
+    return System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV);
+  }
+
+  private String getHttpSchemePrefix() {
+    return HttpConfig.Policy.HTTPS_ONLY ==
+           HttpConfig.Policy.fromString(configuration
+               .get(
+                   YarnConfiguration.YARN_HTTP_POLICY_KEY,
+                   YarnConfiguration.YARN_HTTP_POLICY_DEFAULT))
+           ? HTTPS_PREFIX : HTTP_PREFIX;
+  }
+}