Browse Source

HADOOP-6892. Common component of HDFS-1150 (Verify datanodes' identities to clients in secure clusters).

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@981688 13f79535-47bb-0310-9956-ffa450edef68
Jakob Homan 14 năm trước cách đây
mục cha
commit
1035138b4c
3 tập tin đã thay đổi với 123 bổ sung74 xóa
  1. 3 0
      CHANGES.txt
  2. 13 4
      bin/hadoop-daemon.sh
  3. 107 70
      src/java/org/apache/hadoop/http/HttpServer.java

+ 3 - 0
CHANGES.txt

@@ -30,6 +30,9 @@ Trunk (unreleased changes)
     file systems associated with a particular UGI.  (Devaraj Das and Kan Zhang
     via szetszwo)
 
+    HADOOP-6892. Common component of HDFS-1150 (Verify datanodes' identities 
+    to clients in secure clusters) (jghoman)
+
   IMPROVEMENTS
 
     HADOOP-6644. util.Shell getGROUPS_FOR_USER_COMMAND method name 

+ 13 - 4
bin/hadoop-daemon.sh

@@ -77,20 +77,29 @@ if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
   . "${HADOOP_CONF_DIR}/hadoop-env.sh"
 fi
 
+# Determine if we're starting a secure datanode, and if so, redefine appropriate variables
+if [ "$command" == "datanode" ] && [ "$EUID" -eq 0 ] && [ -n "$HADOOP_SECURE_DN_USER" ]; then
+  export HADOOP_PID_DIR=$HADOOP_SECURE_DN_PID_DIR
+  export HADOOP_LOG_DIR=$HADOOP_SECURE_DN_LOG_DIR
+  export HADOOP_IDENT_STRING=$HADOOP_SECURE_DN_USER   
+fi
+
+if [ "$HADOOP_IDENT_STRING" = "" ]; then
+  export HADOOP_IDENT_STRING="$USER"
+fi
+
+
 # get log directory
 if [ "$HADOOP_LOG_DIR" = "" ]; then
   export HADOOP_LOG_DIR="$HADOOP_HOME/logs"
 fi
 mkdir -p "$HADOOP_LOG_DIR"
+chown $HADOOP_IDENT_STRING $HADOOP_LOG_DIR
 
 if [ "$HADOOP_PID_DIR" = "" ]; then
   HADOOP_PID_DIR=/tmp
 fi
 
-if [ "$HADOOP_IDENT_STRING" = "" ]; then
-  export HADOOP_IDENT_STRING="$USER"
-fi
-
 # some variables
 export HADOOP_LOGFILE=hadoop-$HADOOP_IDENT_STRING-$command-$HOSTNAME.log
 export HADOOP_ROOT_LOGGER="INFO,DRFA"

+ 107 - 70
src/java/org/apache/hadoop/http/HttpServer.java

@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.http;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.io.FileNotFoundException;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.URL;
@@ -43,17 +43,21 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.ConfServlet;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.http.AdminAuthorizedServlet;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.http.FilterInitializer;
+import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.log.LogLevel;
 import org.apache.hadoop.metrics.MetricsServlet;
 import org.apache.hadoop.security.Krb5AndCertsSslSocketConnector;
-import org.apache.hadoop.security.Krb5AndCertsSslSocketConnector.MODE;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.Krb5AndCertsSslSocketConnector.MODE;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.conf.ConfServlet;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-
 import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Handler;
 import org.mortbay.jetty.Server;
@@ -103,12 +107,19 @@ public class HttpServer implements FilterContainer {
   static final String STATE_DESCRIPTION_ALIVE = " - alive";
   static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
 
+  private final boolean listenerStartedExternally;
+  
   /** Same as this(name, bindAddress, port, findPort, null); */
   public HttpServer(String name, String bindAddress, int port, boolean findPort
       ) throws IOException {
     this(name, bindAddress, port, findPort, new Configuration());
   }
 
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf, Connector connector) throws IOException {
+    this(name, bindAddress, port, findPort, conf, null, connector);
+  }
+  
   /**
    * Create a status server on the given port.
    * The jsp scripts are taken from src/webapps/<name>.
@@ -120,9 +131,15 @@ public class HttpServer implements FilterContainer {
    */
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf) throws IOException {
-    this(name, bindAddress, port, findPort, conf, null);
+    this(name, bindAddress, port, findPort, conf, null, null);
   }
 
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf, AccessControlList adminsAcl) 
+      throws IOException {
+    this(name, bindAddress, port, findPort, conf, adminsAcl, null);
+  }
+  
   /**
    * Create a status server on the given port.
    * The jsp scripts are taken from src/webapps/<name>.
@@ -134,14 +151,22 @@ public class HttpServer implements FilterContainer {
    * @param adminsAcl {@link AccessControlList} of the admins
    */
   public HttpServer(String name, String bindAddress, int port,
-      boolean findPort, Configuration conf, AccessControlList adminsAcl)
-      throws IOException {
+      boolean findPort, Configuration conf, AccessControlList adminsAcl, 
+      Connector connector) throws IOException {
     webServer = new Server();
     this.findPort = findPort;
     this.adminsAcl = adminsAcl;
-    listener = createBaseListener(conf);
-    listener.setHost(bindAddress);
-    listener.setPort(port);
+    
+    if(connector == null) {
+      listenerStartedExternally = false;
+      listener = createBaseListener(conf);
+      listener.setHost(bindAddress);
+      listener.setPort(port);
+    } else {
+      listenerStartedExternally = true;
+      listener = connector;
+    }
+    
     webServer.addConnector(listener);
 
     int maxThreads = conf.getInt(HTTP_MAX_THREADS, -1);
@@ -184,8 +209,12 @@ public class HttpServer implements FilterContainer {
    * provided. This wrapper and all subclasses must create at least one
    * listener.
    */
-  protected Connector createBaseListener(Configuration conf)
-      throws IOException {
+  public Connector createBaseListener(Configuration conf) throws IOException {
+    return HttpServer.createDefaultChannelConnector();
+  }
+  
+  @InterfaceAudience.Private
+  public static Connector createDefaultChannelConnector() {
     SelectChannelConnector ret = new SelectChannelConnector();
     ret.setLowResourceMaxIdleTime(10000);
     ret.setAcceptQueueSize(128);
@@ -532,72 +561,80 @@ public class HttpServer implements FilterContainer {
    */
   public void start() throws IOException {
     try {
-      int port = 0;
-      int oriPort = listener.getPort(); // The original requested port
-      while (true) {
-        try {
-          port = webServer.getConnectors()[0].getLocalPort();
-          LOG.info("Port returned by webServer.getConnectors()[0]." +
-          		"getLocalPort() before open() is "+ port + 
-          		". Opening the listener on " + oriPort);
-          listener.open();
-          port = listener.getLocalPort();
-          LOG.info("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(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.
+        webServer.start();
+      } else {
+        int port = 0;
+        int oriPort = listener.getPort(); // The original requested port
+        while (true) {
+          try {
+            port = webServer.getConnectors()[0].getLocalPort();
+            LOG.info("Port returned by webServer.getConnectors()[0]." +
+            		"getLocalPort() before open() is "+ port + 
+            		". Opening the listener on " + oriPort);
+            listener.open();
+            port = listener.getLocalPort();
+            LOG.info("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;
                 }
-                Thread.sleep(200);
+                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();
               }
-              if (port > 0) {
-                break;
+            } //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;
               }
-              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;
             }
-          } else {
-            LOG.info("HttpServer.start() threw a non Bind IOException"); 
+          } catch (MultiException ex) {
+            LOG.info("HttpServer.start() threw a MultiException"); 
             throw ex;
           }
-        } catch (MultiException ex) {
-          LOG.info("HttpServer.start() threw a MultiException"); 
-          throw ex;
+          listener.setPort((oriPort += 1));
         }
-        listener.setPort((oriPort += 1));
       }
     } catch (IOException e) {
       throw e;