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 bo...@apache.org on 2012/04/30 21:00:39 UTC

svn commit: r1332338 - in /hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/http/ src/test/java/org/apache/hadoop/http/

Author: bobby
Date: Mon Apr 30 19:00:38 2012
New Revision: 1332338

URL: http://svn.apache.org/viewvc?rev=1332338&view=rev
Log:
svn merge -c 1332336. FIXES: HADOOP-8334. HttpServer sometimes returns incorrect port (Daryn Sharp via bobby)

Modified:
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1332338&r1=1332337&r2=1332338&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt Mon Apr 30 19:00:38 2012
@@ -44,6 +44,9 @@ Release 0.23.3 - UNRELEASED
 
     HADOOP-8305. distcp over viewfs is broken (John George via bobby)
 
+    HADOOP-8334. HttpServer sometimes returns incorrect port (Daryn Sharp via
+    bobby)
+
 Release 0.23.2 - UNRELEASED 
 
   NEW FEATURES

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java?rev=1332338&r1=1332337&r2=1332338&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java Mon Apr 30 19:00:38 2012
@@ -631,80 +631,16 @@ public class HttpServer implements Filte
    */
   public void start() throws IOException {
     try {
-      if(listenerStartedExternally) { // Expect that listener was started securely
-        if(listener.getLocalPort() == -1) // ... and verify
-          throw new Exception("Exepected webserver's listener to be started " +
-             "previously but wasn't");
-        // And skip all the port rolling issues.
+      try {
+        openListener();
+        LOG.info("Jetty bound to port " + listener.getLocalPort());
         webServer.start();
-      } else {
-        int port = 0;
-        int oriPort = listener.getPort(); // The original requested port
-        while (true) {
-          try {
-            port = webServer.getConnectors()[0].getLocalPort();
-            LOG.debug("Port returned by webServer.getConnectors()[0]." +
-            		"getLocalPort() before open() is "+ port + 
-            		". Opening the listener on " + oriPort);
-            listener.open();
-            port = listener.getLocalPort();
-            LOG.debug("listener.getLocalPort() returned " + listener.getLocalPort() + 
-                  " webServer.getConnectors()[0].getLocalPort() returned " +
-                  webServer.getConnectors()[0].getLocalPort());
-            //Workaround to handle the problem reported in HADOOP-4744
-            if (port < 0) {
-              Thread.sleep(100);
-              int numRetries = 1;
-              while (port < 0) {
-                LOG.warn("listener.getLocalPort returned " + port);
-                if (numRetries++ > MAX_RETRIES) {
-                  throw new Exception(" listener.getLocalPort is returning " +
-                  		"less than 0 even after " +numRetries+" resets");
-                }
-                for (int i = 0; i < 2; i++) {
-                  LOG.info("Retrying listener.getLocalPort()");
-                  port = listener.getLocalPort();
-                  if (port > 0) {
-                    break;
-                  }
-                  Thread.sleep(200);
-                }
-                if (port > 0) {
-                  break;
-                }
-                LOG.info("Bouncing the listener");
-                listener.close();
-                Thread.sleep(1000);
-                listener.setPort(oriPort == 0 ? 0 : (oriPort += 1));
-                listener.open();
-                Thread.sleep(100);
-                port = listener.getLocalPort();
-              }
-            } //Workaround end
-            LOG.info("Jetty bound to port " + port);
-            webServer.start();
-            break;
-          } catch (IOException ex) {
-            // if this is a bind exception,
-            // then try the next port number.
-            if (ex instanceof BindException) {
-              if (!findPort) {
-                BindException be = new BindException(
-                        "Port in use: " + listener.getHost()
-                                + ":" + listener.getPort());
-                be.initCause(ex);
-                throw be;
-              }
-            } else {
-              LOG.info("HttpServer.start() threw a non Bind IOException"); 
-              throw ex;
-            }
-          } catch (MultiException ex) {
-            LOG.info("HttpServer.start() threw a MultiException"); 
-            throw ex;
-          }
-          listener.setPort((oriPort += 1));
-        }
+      } catch (IOException ex) {
+        LOG.info("HttpServer.start() threw a non Bind IOException", ex);
+        throw ex;
+      } catch (MultiException ex) {
+        LOG.info("HttpServer.start() threw a MultiException", ex);
+        throw ex;
       }
     } catch (IOException e) {
       throw e;
@@ -714,6 +650,52 @@ public class HttpServer implements Filte
   }
 
   /**
+   * Open the main listener for the server
+   * @throws Exception
+   */
+  void openListener() throws Exception {
+    if (listener.getLocalPort() != -1) { // it's already bound
+      return;
+    }
+    if (listenerStartedExternally) { // Expect that listener was started securely
+      throw new Exception("Expected webserver's listener to be started " +
+          "previously but wasn't");
+    }
+    int port = listener.getPort();
+    while (true) {
+      // jetty has a bug where you can't reopen a listener that previously
+      // failed to open w/o issuing a close first, even if the port is changed
+      try {
+        listener.close();
+        listener.open();
+        break;
+      } catch (BindException ex) {
+        if (port == 0 || !findPort) {
+          BindException be = new BindException(
+              "Port in use: " + listener.getHost() + ":" + listener.getPort());
+          be.initCause(ex);
+          throw be;
+        }
+      }
+      // try the next port number
+      listener.setPort(++port);
+      Thread.sleep(100);
+    }
+  }
+  
+  /**
+   * Return the bind address of the listener.
+   * @return InetSocketAddress of the listener
+   */
+  public InetSocketAddress getListenerAddress() {
+    int port = listener.getLocalPort();
+    if (port == -1) { // not bound, return requested port
+      port = listener.getPort();
+    }
+    return new InetSocketAddress(listener.getHost(), port);
+  }
+  
+  /**
    * stop the server
    */
   public void stop() throws Exception {

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java?rev=1332338&r1=1332337&r2=1332338&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java Mon Apr 30 19:00:38 2012
@@ -100,6 +100,19 @@ public class HttpServerFunctionalTest ex
   }
 
   /**
+   * Create an HttpServer instance on the given address for the given webapp
+   * @param host to bind
+   * @param port to bind
+   * @return the server
+   * @throws IOException if it could not be created
+   */
+  public static HttpServer createServer(String host, int port)
+      throws IOException {
+    prepareTestWebapp();
+    return new HttpServer(TEST, host, port, true);
+  }
+
+  /**
    * Create an HttpServer instance for the given webapp
    * @param webapp the webapp to work with
    * @return the server

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java?rev=1332338&r1=1332337&r2=1332338&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java Mon Apr 30 19:00:38 2012
@@ -20,6 +20,7 @@ package org.apache.hadoop.http;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
 import java.net.URL;
 import java.util.Arrays;
 import java.util.Enumeration;
@@ -422,4 +423,54 @@ public class TestHttpServer extends Http
     assertEquals("bar", m.get(JerseyResource.OP));
     LOG.info("END testJersey()");
   }
+  @Test public void testBindAddress() throws Exception {
+    checkBindAddress("0.0.0.0", 0, false).stop();
+    // hang onto this one for a bit more testing
+    HttpServer myServer = checkBindAddress("localhost", 0, false);
+    HttpServer myServer2 = null;
+    try { 
+      int port = myServer.getListenerAddress().getPort();
+      // it's already in use, true = expect a higher port
+      myServer2 = checkBindAddress("localhost", port, true);
+      // try to reuse the port
+      port = myServer2.getListenerAddress().getPort();
+      myServer2.stop();
+      assertEquals(-1, myServer2.getPort()); // not bound
+      myServer2.openListener();
+      assertEquals(port, myServer2.getPort()); // expect same port
+    } finally {
+      myServer.stop();
+      if (myServer2 != null) {
+        myServer2.stop();
+      }
+    }
+  }
+  
+  private HttpServer checkBindAddress(String host, int port, boolean findPort)
+      throws Exception {
+    HttpServer server = createServer(host, port);
+    try {
+      // not bound, ephemeral should return requested port (0 for ephemeral)
+      InetSocketAddress addr = server.getListenerAddress();
+      assertEquals(port, addr.getPort());
+      // verify hostname is what was given
+      server.openListener();
+      addr = server.getListenerAddress();
+      assertEquals(host, addr.getHostName());
+
+      int boundPort = addr.getPort();
+      if (port == 0) {
+        assertTrue(boundPort != 0); // ephemeral should now return bound port
+      } else if (findPort) {
+        assertTrue(boundPort > port);
+        // allow a little wiggle room to prevent random test failures if
+        // some consecutive ports are already in use
+        assertTrue(addr.getPort() - port < 8);
+      }
+    } catch (Exception e) {
+      server.stop();
+      throw e;
+    }
+    return server;
+  }
 }