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 dd...@apache.org on 2010/02/25 22:39:39 UTC

svn commit: r916467 - in /hadoop/common/trunk: ./ src/java/ src/java/org/apache/hadoop/conf/ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/http/ src/java/org/apache/hadoop/ipc/ src/java/org/apache/hadoop/log/ src/java/org/apache/hadoop/metr...

Author: ddas
Date: Thu Feb 25 21:39:38 2010
New Revision: 916467

URL: http://svn.apache.org/viewvc?rev=916467&view=rev
Log:
HADOOP-6568. Adds authorization for the default servlets. Contributed by Vinod Kumar Vavilapalli.

Added:
    hadoop/common/trunk/src/java/org/apache/hadoop/http/AdminAuthorizedServlet.java
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/core-default.xml
    hadoop/common/trunk/src/java/org/apache/hadoop/conf/ConfServlet.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
    hadoop/common/trunk/src/java/org/apache/hadoop/http/HttpServer.java
    hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java
    hadoop/common/trunk/src/java/org/apache/hadoop/log/LogLevel.java
    hadoop/common/trunk/src/java/org/apache/hadoop/metrics/MetricsServlet.java
    hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/CLITestHelper.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/http/TestHttpServer.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/ipc/TestRPC.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Thu Feb 25 21:39:38 2010
@@ -62,6 +62,9 @@
     to change the default 1 MB, the maximum size when large IPC handler 
     response buffer is reset. (suresh)
 
+    HADOOP-6568. Adds authorization for the default servlets. 
+    (Vinod Kumar Vavilapalli via ddas)
+
   IMPROVEMENTS
 
     HADOOP-6283. Improve the exception messages thrown by

Modified: hadoop/common/trunk/src/java/core-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/core-default.xml?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/core-default.xml (original)
+++ hadoop/common/trunk/src/java/core-default.xml Thu Feb 25 21:39:38 2010
@@ -54,6 +54,16 @@
 </property>
 
 <property>
+  <name>hadoop.cluster.administrators</name>
+  <property>Users and/or groups who are designated as the administrators of a
+  hadoop cluster. For specifying a list of users and groups the format to use
+  is "user1,user2 group1,group". If set to '*', it allows all users/groups to
+  do administrations operations of the cluster. If set to '', it allows none.
+  </property>
+  <value>${user.name}</value>
+</property>
+
+<property>
   <name>hadoop.security.authorization</name>
   <value>false</value>
   <description>Is service-level authorization enabled?</description>

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/conf/ConfServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/conf/ConfServlet.java?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/conf/ConfServlet.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/conf/ConfServlet.java Thu Feb 25 21:39:38 2010
@@ -52,6 +52,13 @@
   @Override
   public void doGet(HttpServletRequest request, HttpServletResponse response)
       throws ServletException, IOException {
+
+    // Do the authorization
+    if (!HttpServer.hasAdministratorAccess(getServletContext(), request,
+        response)) {
+      return;
+    }
+
     String format = request.getParameter(FORMAT_PARAM);
     if (null == format) {
       format = FORMAT_XML;

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java Thu Feb 25 21:39:38 2010
@@ -133,5 +133,12 @@
   public static final String  HADOOP_SECURITY_GROUP_MAPPING = "hadoop.security.group.mapping";
   public static final String  HADOOP_SECURITY_GROUPS_CACHE_SECS = "hadoop.security.groups.cache.secs";
   public static final String  HADOOP_SECURITY_AUTHENTICATION = "hadoop.security.authentication";
+  public static final String HADOOP_SECURITY_AUTHORIZATION =
+      "hadoop.security.authorization";
+  /**
+   * ACL denoting the administrator ACLs for a hadoop cluster.
+   */
+  public final static String HADOOP_CLUSTER_ADMINISTRATORS_PROPERTY =
+      "hadoop.cluster.administrators";
 }
 

Added: hadoop/common/trunk/src/java/org/apache/hadoop/http/AdminAuthorizedServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/http/AdminAuthorizedServlet.java?rev=916467&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/http/AdminAuthorizedServlet.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/http/AdminAuthorizedServlet.java Thu Feb 25 21:39:38 2010
@@ -0,0 +1,46 @@
+/**
+ * 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.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.mortbay.jetty.servlet.DefaultServlet;
+
+/**
+ * General servlet which is admin-authorized.
+ *
+ */
+public class AdminAuthorizedServlet extends DefaultServlet {
+
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+ throws ServletException, IOException {
+    // Do the authorization
+    if (HttpServer.hasAdministratorAccess(getServletContext(), request,
+        response)) {
+      // Authorization is done. Just call super.
+      super.doGet(request, response);
+    }
+  }
+}

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=916467&r1=916466&r2=916467&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 Thu Feb 25 21:39:38 2010
@@ -27,11 +27,11 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
@@ -45,8 +45,11 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.log.LogLevel;
 import org.apache.hadoop.metrics.MetricsServlet;
+import org.apache.hadoop.security.UserGroupInformation;
+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;
@@ -125,12 +128,13 @@
     webServer.setHandler(contexts);
 
     webAppContext = new WebAppContext();
+    webAppContext.setDisplayName("WepAppsContext");
     webAppContext.setContextPath("/");
     webAppContext.setWar(appDir + "/" + name);
     webAppContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
     webServer.addHandler(webAppContext);
 
-    addDefaultApps(contexts, appDir);
+    addDefaultApps(contexts, appDir, conf);
 
     addGlobalFilter("safety", QuotingInputFilter.class.getName(), null);
     final FilterInitializer[] initializers = getFilterInitializers(conf); 
@@ -182,19 +186,23 @@
    * @throws IOException
    */
   protected void addDefaultApps(ContextHandlerCollection parent,
-      final String appDir) throws IOException {
+      final String appDir, Configuration conf) throws IOException {
     // set up the context for "/logs/" if "hadoop.log.dir" property is defined. 
     String logDir = System.getProperty("hadoop.log.dir");
     if (logDir != null) {
       Context logContext = new Context(parent, "/logs");
       logContext.setResourceBase(logDir);
-      logContext.addServlet(DefaultServlet.class, "/");
+      logContext.addServlet(AdminAuthorizedServlet.class, "/");
+      logContext.setDisplayName("logs");
+      logContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
       defaultContexts.put(logContext, true);
     }
     // set up the context for "/static/*"
     Context staticContext = new Context(parent, "/static");
     staticContext.setResourceBase(appDir + "/static");
     staticContext.addServlet(DefaultServlet.class, "/*");
+    staticContext.setDisplayName("static");
+    staticContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
     defaultContexts.put(staticContext, true);
   }
   
@@ -279,6 +287,8 @@
 
     final String[] USER_FACING_URLS = { "*.html", "*.jsp" };
     defineFilter(webAppContext, name, classname, parameters, USER_FACING_URLS);
+    LOG.info("Added filter " + name + " (class=" + classname
+        + ") to context " + webAppContext.getDisplayName());
     final String[] ALL_URLS = { "/*" };
     for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
       if (e.getValue()) {
@@ -567,6 +577,50 @@
   }
 
   /**
+   * Does the user sending the HttpServletRequest has the administrator ACLs? If
+   * it isn't the case, response will be modified to send an error to the user.
+   * 
+   * @param servletContext
+   * @param request
+   * @param response
+   * @return true if admin-authorized, false otherwise
+   * @throws IOException
+   */
+  public static boolean hasAdministratorAccess(
+      ServletContext servletContext, HttpServletRequest request,
+      HttpServletResponse response) throws IOException {
+    Configuration conf =
+        (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE);
+
+    // If there is no authorization, anybody has administrator access.
+    if (!conf.getBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
+      return true;
+    }
+
+    String remoteUser = request.getRemoteUser();
+    if (remoteUser == null) {
+      return true;
+    }
+
+    String adminsAclString =
+        conf.get(
+            CommonConfigurationKeys.HADOOP_CLUSTER_ADMINISTRATORS_PROPERTY,
+            "*");
+    AccessControlList adminsAcl = new AccessControlList(adminsAclString);
+    UserGroupInformation remoteUserUGI =
+        UserGroupInformation.createRemoteUser(remoteUser);
+    if (!adminsAcl.isUserAllowed(remoteUserUGI)) {
+      response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User "
+          + remoteUser + " is unauthorized to access this page. "
+          + "Only superusers/supergroup \"" + adminsAclString
+          + "\" can access this page.");
+      return false;
+    }
+    return true;
+  }
+
+  /**
    * A very simple servlet to serve up a text representation of the current
    * stack traces. It both returns the stacks to the caller and logs them.
    * Currently the stack traces are done sequentially rather than exactly the
@@ -578,7 +632,13 @@
     @Override
     public void doGet(HttpServletRequest request, HttpServletResponse response)
       throws ServletException, IOException {
-      
+
+      // Do the authorization
+      if (!HttpServer.hasAdministratorAccess(getServletContext(), request,
+          response)) {
+        return;
+      }
+
       PrintWriter out = new PrintWriter
                     (HtmlQuoting.quoteOutputStream(response.getOutputStream()));
       ReflectionUtils.printThreadInfo(out, "");

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java Thu Feb 25 21:39:38 2010
@@ -59,6 +59,8 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+
 import static org.apache.hadoop.fs.CommonConfigurationKeys.*;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
@@ -1295,7 +1297,7 @@
     this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
     this.secretManager = (SecretManager<TokenIdentifier>) secretManager;
     this.authorize = 
-      conf.getBoolean(ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, 
+      conf.getBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, 
                       false);
     this.isSecurityEnabled = UserGroupInformation.isSecurityEnabled();
     

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/log/LogLevel.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/log/LogLevel.java?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/log/LogLevel.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/log/LogLevel.java Thu Feb 25 21:39:38 2010
@@ -26,6 +26,7 @@
 
 import org.apache.commons.logging.*;
 import org.apache.commons.logging.impl.*;
+import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.util.ServletUtil;
 
 /**
@@ -86,6 +87,13 @@
 
     public void doGet(HttpServletRequest request, HttpServletResponse response
         ) throws ServletException, IOException {
+
+      // Do the authorization
+      if (!HttpServer.hasAdministratorAccess(getServletContext(), request,
+          response)) {
+        return;
+      }
+
       PrintWriter out = ServletUtil.initHTML(response, "Log Level");
       String logName = ServletUtil.getParameter(request, "log");
       String level = ServletUtil.getParameter(request, "level");

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/metrics/MetricsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/metrics/MetricsServlet.java?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/metrics/MetricsServlet.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/metrics/MetricsServlet.java Thu Feb 25 21:39:38 2010
@@ -30,6 +30,7 @@
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.metrics.spi.OutputRecord;
 import org.apache.hadoop.metrics.spi.AbstractMetricsContext.MetricMap;
 import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
@@ -100,6 +101,13 @@
   @Override
   public void doGet(HttpServletRequest request, HttpServletResponse response)
       throws ServletException, IOException {
+
+    // Do the authorization
+    if (!HttpServer.hasAdministratorAccess(getServletContext(), request,
+        response)) {
+      return;
+    }
+
     PrintWriter out = new PrintWriter(response.getOutputStream());
     String format = request.getParameter("format");
     Collection<MetricsContext> allContexts = 

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java Thu Feb 25 21:39:38 2010
@@ -21,6 +21,7 @@
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.security.UserGroupInformation;
 
 /**
@@ -35,7 +36,12 @@
   
   /**
    * Configuration key for controlling service-level authorization for Hadoop.
+   * 
+   * @deprecated Use
+   *             {@link CommonConfigurationKeys#HADOOP_SECURITY_AUTHORIZATION}
+   *             Instead.
    */
+  @Deprecated
   public static final String SERVICE_AUTHORIZATION_CONFIG = 
     "hadoop.security.authorization";
   

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/CLITestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/CLITestHelper.java?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/CLITestHelper.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/CLITestHelper.java Thu Feb 25 21:39:38 2010
@@ -28,6 +28,7 @@
 import org.apache.hadoop.cli.util.ComparatorBase;
 import org.apache.hadoop.cli.util.ComparatorData;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.util.StringUtils;
 import static org.junit.Assert.assertTrue;
@@ -104,7 +105,7 @@
     readTestConfigFile();
     
     conf = new Configuration();
-    conf.setBoolean(ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, 
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, 
                     true);
 
     clitestDataDir = new File(TEST_CACHE_DATA_DIR).

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=916467&r1=916466&r2=916467&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 Thu Feb 25 21:39:38 2010
@@ -23,17 +23,35 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
+import java.net.HttpURLConnection;
 import java.net.URL;
+import java.util.Arrays;
 import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
 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.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
 import javax.servlet.http.HttpServletResponse;
 
+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;
@@ -134,4 +152,163 @@
                  readOutput(new URL(baseUrl, "/echomap?a=b&c<=d&a=>")));
   }
 
+  /**
+   * Dummy filter that mimics as an authentication filter. Obtains user identity
+   * from the request parameter user.name. Wraps around the request so that
+   * request.getRemoteUser() returns the user identity.
+   * 
+   */
+  public static class DummyServletFilter implements Filter {
+
+    private static final Log LOG = LogFactory.getLog(
+        DummyServletFilter.class);
+    @Override
+    public void destroy() { }
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response,
+        FilterChain filterChain) throws IOException, ServletException {
+      final String userName = request.getParameter("user.name");
+      ServletRequest requestModified =
+        new HttpServletRequestWrapper((HttpServletRequest) request) {
+        @Override
+        public String getRemoteUser() {
+          return userName;
+        }
+      };
+      filterChain.doFilter(requestModified, response);
+    }
+
+    @Override
+    public void init(FilterConfig arg0) throws ServletException { }
+  }
+
+  /**
+   * FilterInitializer that initialized the DummyFilter.
+   *
+   */
+  public static class DummyFilterInitializer extends FilterInitializer {
+    public DummyFilterInitializer() {
+    }
+
+    @Override
+    public void initFilter(FilterContainer container, Configuration conf) {
+      container.addFilter("DummyFilter", DummyServletFilter.class.getName(), null);
+    }
+  }
+
+  /**
+   * Access a URL and get the corresponding return Http status code. The URL
+   * will be accessed as the passed user, by sending user.name request
+   * parameter.
+   * 
+   * @param urlstring
+   * @param userName
+   * @return
+   * @throws IOException
+   */
+  static int getHttpStatusCode(String urlstring, String userName)
+      throws IOException {
+    URL url = new URL(urlstring + "?user.name=" + userName);
+    System.out.println("Accessing " + url + " as user " + userName);
+    HttpURLConnection connection = (HttpURLConnection)url.openConnection();
+    connection.connect();
+    return connection.getResponseCode();
+  }
+
+  /**
+   * Custom user->group mapping service.
+   */
+  public static class MyGroupsProvider extends ShellBasedUnixGroupsMapping {
+    static Map<String, List<String>> mapping = new HashMap<String, List<String>>();
+
+    static void clearMapping() {
+      mapping.clear();
+    }
+
+    @Override
+    public List<String> getGroups(String user) throws IOException {
+      return mapping.get(user);
+    }
+  }
+
+  /**
+   * Verify the access for /logs, /stacks, /conf, /logLevel and /metrics
+   * servlets, when authentication filters are set, but authorization is not
+   * enabled.
+   * @throws Exception 
+   */
+  @Test
+  public void testDisabledAuthorizationOfDefaultServlets() throws Exception {
+
+    Configuration conf = new Configuration();
+
+    // Authorization is disabled by default
+    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+        DummyFilterInitializer.class.getName());
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+        MyGroupsProvider.class.getName());
+    Groups.getUserToGroupsMappingService(conf);
+    MyGroupsProvider.clearMapping();
+    MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
+    MyGroupsProvider.mapping.put("userB", Arrays.asList("groupB"));
+
+    HttpServer myServer = new HttpServer("test", "0.0.0.0", 0, true, conf);
+    myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
+    myServer.start();
+    int port = myServer.getPort();
+    String serverURL = "http://localhost:" + port + "/";
+    for (String servlet : new String[] { "conf", "logs", "stacks",
+        "logLevel", "metrics" }) {
+      for (String user : new String[] { "userA", "userB" }) {
+        assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(serverURL
+            + servlet, user));
+      }
+    }
+    myServer.stop();
+  }
+
+  /**
+   * Verify the administrator access for /logs, /stacks, /conf, /logLevel and
+   * /metrics servlets.
+   * 
+   * @throws Exception
+   */
+  @Test
+  public void testAuthorizationOfDefaultServlets() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
+        true);
+    conf.set(
+        CommonConfigurationKeys.HADOOP_CLUSTER_ADMINISTRATORS_PROPERTY,
+        "userA,userB groupC,groupD");
+    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+        DummyFilterInitializer.class.getName());
+
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+        MyGroupsProvider.class.getName());
+    Groups.getUserToGroupsMappingService(conf);
+    MyGroupsProvider.clearMapping();
+    MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
+    MyGroupsProvider.mapping.put("userB", Arrays.asList("groupB"));
+    MyGroupsProvider.mapping.put("userC", Arrays.asList("groupC"));
+    MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
+    MyGroupsProvider.mapping.put("userE", Arrays.asList("groupE"));
+
+    HttpServer myServer = new HttpServer("test", "0.0.0.0", 0, true, conf);
+    myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
+    myServer.start();
+    int port = myServer.getPort();
+    String serverURL = "http://localhost:" + port + "/";
+    for (String servlet : new String[] { "conf", "logs", "stacks",
+        "logLevel", "metrics" }) {
+      for (String user : new String[] { "userA", "userB", "userC", "userD" }) {
+        assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(serverURL
+            + servlet, user));
+      }
+      assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, getHttpStatusCode(
+          serverURL + servlet, "userE"));
+    }
+    myServer.stop();
+  }
 }

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/ipc/TestRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/ipc/TestRPC.java?rev=916467&r1=916466&r2=916467&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/ipc/TestRPC.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/ipc/TestRPC.java Thu Feb 25 21:39:38 2010
@@ -30,6 +30,7 @@
 import org.apache.commons.logging.*;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
 
@@ -398,8 +399,8 @@
   
   public void testAuthorization() throws Exception {
     Configuration conf = new Configuration();
-    conf.setBoolean(
-        ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, true);
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
+        true);
     
     // Expect to succeed
     conf.set(ACL_CONFIG, "*");