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

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

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,