You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by vi...@apache.org on 2011/09/20 15:11:12 UTC

svn commit: r1173130 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/http/ src/test/java/org/apache/hadoop/http/

Author: vinodkv
Date: Tue Sep 20 13:11:12 2011
New Revision: 1173130

URL: http://svn.apache.org/viewvc?rev=1173130&view=rev
Log:
HADOOP-7639. Enhance HttpServer to allow passing path-specs for filtering, so that servers like Yarn WebApp can get filtered the paths served by their own injected servlets. Contributed by Thomas Graves.

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1173130&r1=1173129&r2=1173130&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Tue Sep 20 13:11:12 2011
@@ -388,6 +388,10 @@ Release 0.23.0 - Unreleased
     HADOOP-7599. Script improvements to setup a secure Hadoop cluster
     (Eric Yang via ddas)
 
+    HADOOP-7639. Enhance HttpServer to allow passing path-specs for filtering,
+    so that servers like Yarn WebApp can get filtered the paths served by
+    their own injected servlets. (Thomas Graves via vinodkv)
+
   OPTIMIZATIONS
   
     HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java?rev=1173130&r1=1173129&r2=1173130&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java Tue Sep 20 13:11:12 2011
@@ -124,6 +124,29 @@ public class HttpServer implements Filte
       boolean findPort, Configuration conf, Connector connector) throws IOException {
     this(name, bindAddress, port, findPort, conf, null, connector);
   }
+
+  /**
+   * Create a status server on the given port. Allows you to specify the
+   * path specifications that this server will be serving so that they will be
+   * added to the filters properly.  
+   * 
+   * @param name The name of the server
+   * @param bindAddress The address for this server
+   * @param port The port to use on the server
+   * @param findPort whether the server should start at the given port and 
+   *        increment by 1 until it finds a free port.
+   * @param conf Configuration 
+   * @param pathSpecs Path specifications that this httpserver will be serving. 
+   *        These will be added to any filters.
+   */
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
+    this(name, bindAddress, port, findPort, conf, null, null);
+    for (String path : pathSpecs) {
+        LOG.info("adding path spec: " + path);
+      addFilterPathMapping(path, webAppContext);
+    }
+  }
   
   /**
    * Create a status server on the given port.

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java?rev=1173130&r1=1173129&r2=1173130&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java Tue Sep 20 13:11:12 2011
@@ -71,6 +71,21 @@ public class HttpServerFunctionalTest ex
   }
 
   /**
+   * Create but do not start the test webapp server. The test webapp dir is
+   * prepared/checked in advance.
+   * @param conf the server configuration to use
+   * @return the server instance
+   *
+   * @throws IOException if a problem occurs
+   * @throws AssertionError if a condition was not met
+   */
+  public static HttpServer createTestServer(Configuration conf, 
+      String[] pathSpecs) throws IOException {
+    prepareTestWebapp();
+    return createServer(TEST, conf, pathSpecs);
+  }
+
+  /**
    * Prepare the test webapp by creating the directory from the test properties
    * fail if the directory cannot be created.
    * @throws AssertionError if a condition was not met
@@ -104,6 +119,18 @@ public class HttpServerFunctionalTest ex
       throws IOException {
     return new HttpServer(webapp, "0.0.0.0", 0, true, conf);
   }
+  /**
+   * Create an HttpServer instance for the given webapp
+   * @param webapp the webapp to work with
+   * @param conf the configuration to use for the server
+   * @param pathSpecs the paths specifications the server will service
+   * @return the server
+   * @throws IOException if it could not be created
+   */
+  public static HttpServer createServer(String webapp, Configuration conf,
+      String[] pathSpecs) throws IOException {
+    return new HttpServer(webapp, "0.0.0.0", 0, true, conf, pathSpecs);
+  }
 
   /**
    * Create and start a server with the test webapp

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java?rev=1173130&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java Tue Sep 20 13:11:12 2011
@@ -0,0 +1,145 @@
+/**
+ * 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.hadoop.http;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.Set;
+import java.util.TreeSet;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class TestPathFilter extends HttpServerFunctionalTest {
+  static final Log LOG = LogFactory.getLog(HttpServer.class);
+  static final Set<String> RECORDS = new TreeSet<String>(); 
+
+  /** A very simple filter that records accessed uri's */
+  static public class RecordingFilter implements Filter {
+    private FilterConfig filterConfig = null;
+
+    public void init(FilterConfig filterConfig) {
+      this.filterConfig = filterConfig;
+    }
+
+    public void destroy() {
+      this.filterConfig = null;
+    }
+
+    public void doFilter(ServletRequest request, ServletResponse response,
+        FilterChain chain) throws IOException, ServletException {
+      if (filterConfig == null)
+         return;
+
+      String uri = ((HttpServletRequest)request).getRequestURI();
+      LOG.info("filtering " + uri);
+      RECORDS.add(uri);
+      chain.doFilter(request, response);
+    }
+
+    /** Configuration for RecordingFilter */
+    static public class Initializer extends FilterInitializer {
+      public Initializer() {}
+
+      public void initFilter(FilterContainer container, Configuration conf) {
+        container.addFilter("recording", RecordingFilter.class.getName(), null);
+      }
+    }
+  }
+  
+  
+  /** access a url, ignoring some IOException such as the page does not exist */
+  static void access(String urlstring) throws IOException {
+    LOG.warn("access " + urlstring);
+    URL url = new URL(urlstring);
+    
+    URLConnection connection = url.openConnection();
+    connection.connect();
+    
+    try {
+      BufferedReader in = new BufferedReader(new InputStreamReader(
+          connection.getInputStream()));
+      try {
+        for(; in.readLine() != null; );
+      } finally {
+        in.close();
+      }
+    } catch(IOException ioe) {
+      LOG.warn("urlstring=" + urlstring, ioe);
+    }
+  }
+
+  @Test
+  public void testPathSpecFilters() throws Exception {
+    Configuration conf = new Configuration();
+    
+    //start a http server with CountingFilter
+    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+        RecordingFilter.Initializer.class.getName());
+    String[] pathSpecs = { "/path", "/path/*" };
+    HttpServer http = createTestServer(conf, pathSpecs);
+    http.start();
+
+    final String baseURL = "/path";
+    final String baseSlashURL = "/path/";
+    final String addedURL = "/path/nodes";
+    final String addedSlashURL = "/path/nodes/";
+    final String longURL = "/path/nodes/foo/job";
+    final String rootURL = "/";
+    final String allURL = "/*";
+
+    final String[] filteredUrls = {baseURL, baseSlashURL, addedURL, 
+        addedSlashURL, longURL};
+    final String[] notFilteredUrls = {rootURL, allURL};
+
+    // access the urls and verify our paths specs got added to the 
+    // filters
+    final String prefix = "http://localhost:" + http.getPort();
+    try {
+      for(int i = 0; i < filteredUrls.length; i++) {
+        access(prefix + filteredUrls[i]);
+      }
+      for(int i = 0; i < notFilteredUrls.length; i++) {
+        access(prefix + notFilteredUrls[i]);
+      }
+    } finally {
+      http.stop();
+    }
+
+    LOG.info("RECORDS = " + RECORDS);
+    
+    //verify records
+    for(int i = 0; i < filteredUrls.length; i++) {
+      assertTrue(RECORDS.remove(filteredUrls[i]));
+    }
+    assertTrue(RECORDS.isEmpty());
+  }
+}