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 to...@apache.org on 2010/06/02 17:43:51 UTC

svn commit: r950605 - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/http/ src/test/core/org/apache/hadoop/http/

Author: tomwhite
Date: Wed Jun  2 15:43:51 2010
New Revision: 950605

URL: http://svn.apache.org/viewvc?rev=950605&view=rev
Log:
HADOOP-6461.  Webapps aren't located correctly post-split.  Contributed by Todd Lipcon and Steve Loughran.

Added:
    hadoop/common/trunk/src/test/core/org/apache/hadoop/http/HttpServerFunctionalTest.java   (with props)
    hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerWebapps.java   (with props)
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/build.xml
    hadoop/common/trunk/src/java/org/apache/hadoop/http/HttpServer.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestGlobalFilter.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServer.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerLifecycle.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestServletFilter.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=950605&r1=950604&r2=950605&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Wed Jun  2 15:43:51 2010
@@ -1602,6 +1602,9 @@ Release 0.21.0 - Unreleased
     HADOOP-6404. Rename the generated artifacts to common instead of core.
     (tomwhite)
 
+    HADOOP-6461.  Webapps aren't located correctly post-split.
+    (Todd Lipcon and Steve Loughran via tomwhite)
+
 Release 0.20.3 - Unreleased
 
   NEW FEATURES

Modified: hadoop/common/trunk/build.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/build.xml?rev=950605&r1=950604&r2=950605&view=diff
==============================================================================
--- hadoop/common/trunk/build.xml (original)
+++ hadoop/common/trunk/build.xml Wed Jun  2 15:43:51 2010
@@ -79,6 +79,7 @@
   <property name="test.build.extraconf" value="${test.build.dir}/extraconf"/>
   <property name="test.build.javadoc" value="${test.build.dir}/docs/api"/>
   <property name="test.build.javadoc.dev" value="${test.build.dir}/docs/dev-api"/>
+  <property name="test.build.webapps" value="${build.dir}/webapps"/>
   <property name="test.include" value="Test*"/>
   <property name="test.classpath.id" value="test.classpath"/>
   <property name="test.output" value="no"/>
@@ -630,6 +631,8 @@
       <delete file="${test.build.dir}/testsfailed"/>
       <delete dir="@{test.dir}/data" />
       <mkdir dir="@{test.dir}/data" />
+      <delete dir="${test.build.webapps}"/>
+      <mkdir dir="${test.build.webapps}/test" />
       <delete dir="@{test.dir}/logs" />
       <mkdir dir="@{test.dir}/logs" />
       <copy file="${test.src.dir}/hadoop-policy.xml"

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/http/HttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/http/HttpServer.java?rev=950605&r1=950604&r2=950605&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/http/HttpServer.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/http/HttpServer.java Wed Jun  2 15:43:51 2010
@@ -19,6 +19,7 @@ package org.apache.hadoop.http;
 
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.io.FileNotFoundException;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.URL;
@@ -129,7 +130,7 @@ public class HttpServer implements Filte
         new QueuedThreadPool() : new QueuedThreadPool(maxThreads);
     webServer.setThreadPool(threadPool);
 
-    final String appDir = getWebAppsPath();
+    final String appDir = getWebAppsPath(name);
     ContextHandlerCollection contexts = new ContextHandlerCollection();
     webServer.setHandler(contexts);
 
@@ -364,14 +365,17 @@ public class HttpServer implements Filte
 
   /**
    * Get the pathname to the webapps files.
+   * @param appName eg "secondary" or "datanode"
    * @return the pathname as a URL
-   * @throws IOException if 'webapps' directory cannot be found on CLASSPATH.
+   * @throws FileNotFoundException if 'webapps' directory cannot be found on CLASSPATH.
    */
-  protected String getWebAppsPath() throws IOException {
-    URL url = getClass().getClassLoader().getResource("webapps");
+  private String getWebAppsPath(String appName) throws FileNotFoundException {
+    URL url = getClass().getClassLoader().getResource("webapps/" + appName);
     if (url == null) 
-      throw new IOException("webapps not found in CLASSPATH"); 
-    return url.toString();
+      throw new FileNotFoundException("webapps/" + appName
+          + " not found in CLASSPATH");
+    String urlString = url.toString();
+    return urlString.substring(0, urlString.lastIndexOf('/'));
   }
 
   /**

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/http/HttpServerFunctionalTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/http/HttpServerFunctionalTest.java?rev=950605&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/http/HttpServerFunctionalTest.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/http/HttpServerFunctionalTest.java Wed Jun  2 15:43:51 2010
@@ -0,0 +1,167 @@
+/**
+ * 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 org.junit.Assert;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.net.MalformedURLException;
+
+/**
+ * This is a base class for functional tests of the {@link HttpServer}.
+ * The methods are static for other classes to import statically.
+ */
+public class HttpServerFunctionalTest extends Assert {
+  /** JVM property for the webapp test dir : {@value} */
+  public static final String TEST_BUILD_WEBAPPS = "test.build.webapps";
+  /** expected location of the test.build.webapps dir: {@value} */
+  private static final String BUILD_WEBAPPS_DIR = "build/webapps";
+  
+  /** name of the test webapp: {@value} */
+  private static final String TEST = "test";
+
+  /**
+   * Create but do not start the test webapp server. The test webapp dir is
+   * prepared/checked in advance.
+   *
+   * @return the server instance
+   *
+   * @throws IOException if a problem occurs
+   * @throws AssertionError if a condition was not met
+   */
+  public static HttpServer createTestServer() throws IOException {
+    prepareTestWebapp();
+    return createServer(TEST);
+  }
+
+  /**
+   * 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)
+      throws IOException {
+    prepareTestWebapp();
+    return createServer(TEST, conf);
+  }
+
+  /**
+   * 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
+   */
+  protected static void prepareTestWebapp() {
+    String webapps = System.getProperty(TEST_BUILD_WEBAPPS, BUILD_WEBAPPS_DIR);
+    File testWebappDir = new File(webapps +
+        File.pathSeparator + TEST);
+    if (!testWebappDir.exists()) {
+      assertTrue("Unable to create the test dir " + testWebappDir,
+          testWebappDir.mkdirs());
+    } else {
+      assertTrue("Not a directory " + testWebappDir,
+          testWebappDir.isDirectory());
+    }
+  }
+
+  /**
+   * Create an HttpServer instance for the given webapp
+   * @param webapp the webapp to work with
+   * @return the server
+   * @throws IOException if it could not be created
+   */
+  public static HttpServer createServer(String webapp) throws IOException {
+    return new HttpServer(webapp, "0.0.0.0", 0, true);
+  }
+  /**
+   * Create an HttpServer instance for the given webapp
+   * @param webapp the webapp to work with
+   * @param conf the configuration to use for the server
+   * @return the server
+   * @throws IOException if it could not be created
+   */
+  public static HttpServer createServer(String webapp, Configuration conf)
+      throws IOException {
+    return new HttpServer(webapp, "0.0.0.0", 0, true, conf);
+  }
+
+  /**
+   * Create and start a server with the test webapp
+   *
+   * @return the newly started server
+   *
+   * @throws IOException on any failure
+   * @throws AssertionError if a condition was not met
+   */
+  public static HttpServer createAndStartTestServer() throws IOException {
+    HttpServer server = createTestServer();
+    server.start();
+    return server;
+  }
+
+  /**
+   * If the server is non null, stop it
+   * @param server to stop
+   * @throws Exception on any failure
+   */
+  public static void stop(HttpServer server) throws Exception {
+    if (server != null) {
+      server.stop();
+    }
+  }
+
+  /**
+   * Pass in a server, return a URL bound to localhost and its port
+   * @param server server
+   * @return a URL bonded to the base of the server
+   * @throws MalformedURLException if the URL cannot be created.
+   */
+  public static URL getServerURL(HttpServer server)
+      throws MalformedURLException {
+    assertNotNull("No server", server);
+    int port = server.getPort();
+    return new URL("http://localhost:" + port + "/");
+  }
+
+  /**
+   * Read in the content from a URL
+   * @param url URL To read
+   * @return the text from the output
+   * @throws IOException if something went wrong
+   */
+  protected static String readOutput(URL url) throws IOException {
+    StringBuilder out = new StringBuilder();
+    InputStream in = url.openConnection().getInputStream();
+    byte[] buffer = new byte[64 * 1024];
+    int len = in.read(buffer);
+    while (len > 0) {
+      out.append(new String(buffer, 0, len));
+      len = in.read(buffer);
+    }
+    return out.toString();
+  }
+}

Propchange: hadoop/common/trunk/src/test/core/org/apache/hadoop/http/HttpServerFunctionalTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestGlobalFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestGlobalFilter.java?rev=950605&r1=950604&r2=950605&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestGlobalFilter.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestGlobalFilter.java Wed Jun  2 15:43:51 2010
@@ -36,8 +36,9 @@ import javax.servlet.http.HttpServletReq
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
 
-public class TestGlobalFilter extends junit.framework.TestCase {
+public class TestGlobalFilter extends HttpServerFunctionalTest {
   static final Log LOG = LogFactory.getLog(HttpServer.class);
   static final Set<String> RECORDS = new TreeSet<String>(); 
 
@@ -95,13 +96,14 @@ public class TestGlobalFilter extends ju
     }
   }
 
+  @Test
   public void testServletFilter() throws Exception {
     Configuration conf = new Configuration();
     
     //start a http server with CountingFilter
     conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
         RecordingFilter.Initializer.class.getName());
-    HttpServer http = new HttpServer("..", "localhost", 0, true, conf);
+    HttpServer http = createTestServer(conf);
     http.start();
 
     final String fsckURL = "/fsck";

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServer.java?rev=950605&r1=950604&r2=950605&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServer.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServer.java Wed Jun  2 15:43:51 2010
@@ -17,12 +17,7 @@
  */
 package org.apache.hadoop.http;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.PrintStream;
 import java.net.HttpURLConnection;
 import java.net.URL;
@@ -50,17 +45,15 @@ import javax.servlet.http.HttpServletRes
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.ConfServlet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.http.HttpServer.QuotingInputFilter;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class TestHttpServer {
+public class TestHttpServer extends HttpServerFunctionalTest {
   private static HttpServer server;
   private static URL baseUrl;
   private static final int MAX_THREADS = 10;
@@ -115,30 +108,12 @@ public class TestHttpServer {
     }    
   }
 
-  private String readOutput(URL url) throws IOException {
-    StringBuilder out = new StringBuilder();
-    InputStream in = url.openConnection().getInputStream();
-    byte[] buffer = new byte[64 * 1024];
-    int len = in.read(buffer);
-    while (len > 0) {
-      out.append(new String(buffer, 0, len));
-      len = in.read(buffer);
-    }
-    return out.toString();
-  }
-  
   @BeforeClass public static void setup() throws Exception {
-    new File(System.getProperty("build.webapps", "build/webapps") + "/test"
-             ).mkdirs();
-    Configuration conf = new Configuration();
-    // Set the maximum threads
-    conf.setInt(HttpServer.HTTP_MAX_THREADS, MAX_THREADS);
-    server = new HttpServer("test", "0.0.0.0", 0, true, conf);
+    server = createTestServer();
     server.addServlet("echo", "/echo", EchoServlet.class);
     server.addServlet("echomap", "/echomap", EchoMapServlet.class);
     server.start();
-    int port = server.getPort();
-    baseUrl = new URL("http://localhost:" + port + "/");
+    baseUrl = getServerURL(server);
   }
   
   @AfterClass public static void cleanup() throws Exception {

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerLifecycle.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerLifecycle.java?rev=950605&r1=950604&r2=950605&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerLifecycle.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerLifecycle.java Wed Jun  2 15:43:51 2010
@@ -17,46 +17,9 @@
  */
 package org.apache.hadoop.http;
 
-import static org.junit.Assert.assertTrue;
 import org.junit.Test;
 
-import java.io.File;
-
-public class TestHttpServerLifecycle {
-
-
-  /**
-   * Create but do not start the server
-   * @return the server instance in the member variable "server"
-   * @throws Exception on any failure
-   */
-  private HttpServer createServer() throws Exception {
-    new File(System.getProperty("build.webapps", "build/webapps") + "/test"
-             ).mkdirs();
-    HttpServer server = new HttpServer("test", "0.0.0.0", 0, true);
-    return server;
-  }
-
-  /**
-   * Create and start the server
-   * @return the newly started server
-   * @throws Exception on any failure
-   */
-  private HttpServer createAndStartServer() throws Exception {
-    HttpServer server = createServer();
-    server.start();
-    return server;
-  }
-
-  /**
-   * If the server is non null, stop it
-   * @throws Exception on any failure
-   */
-  private void stop(HttpServer server) throws Exception {
-    if (server != null) {
-      server.stop();
-    }
-  }
+public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
 
   /**
    * Check that a server is alive by probing the {@link HttpServer#isAlive()} method
@@ -79,12 +42,12 @@ public class TestHttpServerLifecycle {
    * @throws Throwable on failure
    */
   @Test public void testCreatedServerIsNotAlive() throws Throwable {
-    HttpServer server = createServer();
+    HttpServer server = createTestServer();
     assertNotLive(server);
   }
 
   @Test public void testStopUnstartedServer() throws Throwable {
-    HttpServer server = createServer();
+    HttpServer server = createTestServer();
     stop(server);
   }
 
@@ -96,7 +59,7 @@ public class TestHttpServerLifecycle {
   @Test public void testStartedServerIsAlive() throws Throwable {
     HttpServer server = null;
     try {
-      server = createServer();
+      server = createTestServer();
       assertNotLive(server);
       server.start();
       assertAlive(server);
@@ -122,7 +85,7 @@ public class TestHttpServerLifecycle {
    * @throws Throwable on failure
    */
   @Test public void testStoppedServerIsNotAlive() throws Throwable {
-    HttpServer server = createAndStartServer();
+    HttpServer server = createAndStartTestServer();
     assertAlive(server);
     stop(server);
     assertNotLive(server);
@@ -134,7 +97,7 @@ public class TestHttpServerLifecycle {
    * @throws Throwable on failure
    */
   @Test public void testStoppingTwiceServerIsAllowed() throws Throwable {
-    HttpServer server = createAndStartServer();
+    HttpServer server = createAndStartTestServer();
     assertAlive(server);
     stop(server);
     assertNotLive(server);

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerWebapps.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerWebapps.java?rev=950605&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerWebapps.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerWebapps.java Wed Jun  2 15:43:51 2010
@@ -0,0 +1,65 @@
+/**
+ * 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 org.junit.Test;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+
+import java.io.FileNotFoundException;
+
+/**
+ * Test webapp loading
+ */
+public class TestHttpServerWebapps extends HttpServerFunctionalTest {
+  private static final Log log = LogFactory.getLog(TestHttpServerWebapps.class);
+
+  /**
+   * Test that the test server is loadable on the classpath
+   * @throws Throwable if something went wrong
+   */
+  @Test
+  public void testValidServerResource() throws Throwable {
+    HttpServer server = null;
+    try {
+      server = createServer("test");
+    } finally {
+      stop(server);
+    }
+  }
+
+  /**
+   * Test that an invalid webapp triggers an exception
+   * @throws Throwable if something went wrong
+   */
+  @Test
+  public void testMissingServerResource() throws Throwable {
+    try {
+      HttpServer server = createServer("NoSuchWebapp");
+      //should not have got here.
+      //close the server
+      String serverDescription = server.toString();
+      stop(server);
+      fail("Expected an exception, got " + serverDescription);
+    } catch (FileNotFoundException expected) {
+      log.debug("Expected exception " + expected, expected);
+    }
+  }
+
+}

Propchange: hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServerWebapps.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestServletFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestServletFilter.java?rev=950605&r1=950604&r2=950605&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestServletFilter.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestServletFilter.java Wed Jun  2 15:43:51 2010
@@ -35,8 +35,9 @@ import javax.servlet.http.HttpServletReq
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
 
-public class TestServletFilter extends junit.framework.TestCase {
+public class TestServletFilter extends HttpServerFunctionalTest {
   static final Log LOG = LogFactory.getLog(HttpServer.class);
   static volatile String uri = null; 
 
@@ -93,13 +94,14 @@ public class TestServletFilter extends j
     }
   }
 
+  @Test
   public void testServletFilter() throws Exception {
     Configuration conf = new Configuration();
     
     //start a http server with CountingFilter
     conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
         SimpleFilter.Initializer.class.getName());
-    HttpServer http = new HttpServer("..", "localhost", 0, true, conf);
+    HttpServer http = createTestServer(conf);
     http.start();
 
     final String fsckURL = "/fsck";