Sfoglia il codice sorgente

commit b003831d6c1e48073b77a50aab0ad3bbc2138315
Author: Vinod Kumar <vinodkv@yahoo-inc.com>
Date: Fri May 7 11:52:42 2010 +0530

MAPREDUCE-1754 and HADOOP-6748 from https://issues.apache.org/jira/secure/attachment/12443928/patch-1754-ydist.txt.

+++ b/YAHOO-CHANGES.txt
+
+ MAPREDUCE-1754. Replace mapred.persmissions.supergroup with an acl :
+ mapreduce.cluster.administrators and HADOOP-6748.: Remove
+ hadoop.cluster.administrators. Contributed by Amareshwari Sriramadasu.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1077446 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 anni fa
parent
commit
0916752147

+ 6 - 6
conf/mapred-queue-acls.xml.template

@@ -18,9 +18,9 @@
     It is only used if authorization is enabled in Map/Reduce by setting the
     It is only used if authorization is enabled in Map/Reduce by setting the
     configuration property mapred.acls.enabled to true.
     configuration property mapred.acls.enabled to true.
 
 
-    Irrespective of this ACL configuration, the user who started the cluster,
-    members of supergroup configured on JobTracker via
-    mapred.permissions.supergroup can submit jobs.
+    Irrespective of this ACL configuration, the user who started the cluster and
+    cluster administrators configured on JobTracker via
+    mapreduce.cluster.administrators can submit jobs.
   </description>
   </description>
 </property>
 </property>
 
 
@@ -38,9 +38,9 @@
     It is only used if authorization is enabled in Map/Reduce by setting the
     It is only used if authorization is enabled in Map/Reduce by setting the
     configuration property mapred.acls.enabled to true.
     configuration property mapred.acls.enabled to true.
 
 
-    Irrespective of this ACL configuration, the user who started the cluster,
-    members of supergroup configured on JobTracker via
-    mapred.permissions.supergroup can do this operation.
+    Irrespective of this ACL configuration, the user who started the cluster and
+    cluster administrators configured on JobTracker via
+    mapreduce.cluster.administrators can do this operation.
   </description>
   </description>
 </property>
 </property>
 
 

+ 0 - 10
src/core/core-default.xml

@@ -31,16 +31,6 @@
   ordering of the filters.</description>
   ordering of the filters.</description>
 </property>
 </property>
 
 
-<property>
-  <name>hadoop.cluster.administrators</name>
-  <value>${user.name}</value>
-  <description>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,group2". If set to '*', it allows all users/groups to
-  do administrative operations of the cluster. If set to '', it allows none.
-  </description>
-</property>
-
 <property>
 <property>
   <name>hadoop.security.authorization</name>
   <name>hadoop.security.authorization</name>
   <value>false</value>
   <value>false</value>

+ 0 - 5
src/core/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -133,10 +133,5 @@ public class CommonConfigurationKeys {
       "hadoop.security.authorization";
       "hadoop.security.authorization";
   public static final String  HADOOP_SECURITY_SERVICE_USER_NAME_KEY = 
   public static final String  HADOOP_SECURITY_SERVICE_USER_NAME_KEY = 
       "hadoop.security.service.user.name.key";
       "hadoop.security.service.user.name.key";
-  /**
-   * ACL denoting the administrator ACLs for a hadoop cluster.
-   */
-  public final static String HADOOP_CLUSTER_ADMINISTRATORS_PROPERTY =
-      "hadoop.cluster.administrators";
 }
 }
 
 

+ 30 - 15
src/core/org/apache/hadoop/http/HttpServer.java

@@ -84,6 +84,9 @@ public class HttpServer implements FilterContainer {
   // The ServletContext attribute where the daemon Configuration
   // The ServletContext attribute where the daemon Configuration
   // gets stored.
   // gets stored.
   static final String CONF_CONTEXT_ATTRIBUTE = "hadoop.conf";
   static final String CONF_CONTEXT_ATTRIBUTE = "hadoop.conf";
+  static final String ADMINS_ACL = "admins.acl";
+
+  private AccessControlList adminsAcl;
 
 
   protected final Server webServer;
   protected final Server webServer;
   protected final Connector listener;
   protected final Connector listener;
@@ -101,6 +104,11 @@ public class HttpServer implements FilterContainer {
     this(name, bindAddress, port, findPort, new Configuration());
     this(name, bindAddress, port, findPort, new Configuration());
   }
   }
 
 
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf) throws IOException {
+    this(name, bindAddress, port, findPort, conf, null);
+  }
+
   /**
   /**
    * Create a status server on the given port.
    * Create a status server on the given port.
    * The jsp scripts are taken from src/webapps/<name>.
    * The jsp scripts are taken from src/webapps/<name>.
@@ -109,12 +117,15 @@ public class HttpServer implements FilterContainer {
    * @param findPort whether the server should start at the given port and 
    * @param findPort whether the server should start at the given port and 
    *        increment by 1 until it finds a free port.
    *        increment by 1 until it finds a free port.
    * @param conf Configuration 
    * @param conf Configuration 
+   * @param adminsAcl {@link AccessControlList} of the admins
    */
    */
   public HttpServer(String name, String bindAddress, int port,
   public HttpServer(String name, String bindAddress, int port,
-      boolean findPort, Configuration conf) throws IOException {
+      boolean findPort, Configuration conf, AccessControlList adminsAcl)
+      throws IOException {
     webServer = new Server();
     webServer = new Server();
     this.findPort = findPort;
     this.findPort = findPort;
     this.conf = conf;
     this.conf = conf;
+    this.adminsAcl = adminsAcl;
 
 
     listener = createBaseListener(conf);
     listener = createBaseListener(conf);
     listener.setHost(bindAddress);
     listener.setHost(bindAddress);
@@ -132,6 +143,7 @@ public class HttpServer implements FilterContainer {
     webAppContext.setContextPath("/");
     webAppContext.setContextPath("/");
     webAppContext.setWar(appDir + "/" + name);
     webAppContext.setWar(appDir + "/" + name);
     webAppContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
     webAppContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
+    webAppContext.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
     webServer.addHandler(webAppContext);
     webServer.addHandler(webAppContext);
 
 
     addDefaultApps(contexts, appDir);
     addDefaultApps(contexts, appDir);
@@ -198,7 +210,7 @@ public class HttpServer implements FilterContainer {
       logContext.setResourceBase(logDir);
       logContext.setResourceBase(logDir);
       logContext.addServlet(AdminAuthorizedServlet.class, "/");
       logContext.addServlet(AdminAuthorizedServlet.class, "/");
       logContext.setDisplayName("logs");
       logContext.setDisplayName("logs");
-      logContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
+      setContextAttributes(logContext);
       defaultContexts.put(logContext, true);
       defaultContexts.put(logContext, true);
     }
     }
     // set up the context for "/static/*"
     // set up the context for "/static/*"
@@ -206,10 +218,15 @@ public class HttpServer implements FilterContainer {
     staticContext.setResourceBase(appDir + "/static");
     staticContext.setResourceBase(appDir + "/static");
     staticContext.addServlet(DefaultServlet.class, "/*");
     staticContext.addServlet(DefaultServlet.class, "/*");
     staticContext.setDisplayName("static");
     staticContext.setDisplayName("static");
-    staticContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
+    setContextAttributes(staticContext);
     defaultContexts.put(staticContext, true);
     defaultContexts.put(staticContext, true);
   }
   }
   
   
+  private void setContextAttributes(Context context) {
+    context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
+    context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
+  }
+
   /**
   /**
    * Add default servlets.
    * Add default servlets.
    */
    */
@@ -625,20 +642,18 @@ public class HttpServer implements FilterContainer {
     if (remoteUser == null) {
     if (remoteUser == null) {
       return true;
       return true;
     }
     }
-
-    String adminsAclString =
-        conf.get(
-            CommonConfigurationKeys.HADOOP_CLUSTER_ADMINISTRATORS_PROPERTY,
-            "*");
-    AccessControlList adminsAcl = new AccessControlList(adminsAclString);
+    AccessControlList adminsAcl = (AccessControlList) servletContext
+        .getAttribute(ADMINS_ACL);
     UserGroupInformation remoteUserUGI =
     UserGroupInformation remoteUserUGI =
         UserGroupInformation.createRemoteUser(remoteUser);
         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;
+    if (adminsAcl != null) {
+      if (!adminsAcl.isUserAllowed(remoteUserUGI)) {
+        response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User "
+            + remoteUser + " is unauthorized to access this page. "
+            + "Only \"" + adminsAcl.toString()
+            + "\" can access this page.");
+        return false;
+      }
     }
     }
     return true;
     return true;
   }
   }

+ 4 - 1
src/core/org/apache/hadoop/security/authorize/AccessControlList.java

@@ -17,7 +17,6 @@
  */
  */
 package org.apache.hadoop.security.authorize;
 package org.apache.hadoop.security.authorize;
 
 
-import java.util.Iterator;
 import java.util.Set;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.TreeSet;
 
 
@@ -76,6 +75,10 @@ public class AccessControlList {
     return allAllowed;
     return allAllowed;
   }
   }
   
   
+  public void addUser(String user) {
+    users.add(user);
+  }
+
   /**
   /**
    * Get the names of users allowed for this service.
    * Get the names of users allowed for this service.
    * @return the set of user names. the set must not be modified.
    * @return the set of user names. the set must not be modified.

+ 2 - 2
src/docs/src/documentation/content/xdocs/mapred_tutorial.xml

@@ -1524,8 +1524,8 @@
           nobody is given access in these properties.</p> 
           nobody is given access in these properties.</p> 
           
           
           <p>However, irrespective of the ACLs configured, a job's owner,
           <p>However, irrespective of the ACLs configured, a job's owner,
-          the superuser and the members of an admin configured supergroup
-          (<code>mapred.permissions.supergroup</code>) always have access to
+          the superuser and cluster administrators
+          (<code>mapreduce.cluster.administrators</code>) always have access to
           view and modify a job.</p>
           view and modify a job.</p>
           
           
           <p> A job view ACL authorizes users against the configured 
           <p> A job view ACL authorizes users against the configured 

+ 10 - 8
src/mapred/mapred-default.xml

@@ -942,13 +942,15 @@
     have the authorization to satisfy either the queue-level ACL or the
     have the authorization to satisfy either the queue-level ACL or the
     job-level ACL.
     job-level ACL.
 
 
-    Irrespective of this ACL configuration, job-owner, superuser, members
-    of supergroup configured on JobTracker via mapred.permissions.supergroup
+    Irrespective of this ACL configuration, job-owner, the user who started the
+    cluster, cluster administrators configured on JobTracker via 
+    mapreduce.cluster.administrators
     and administrators of the queue to which this job is submitted to 
     and administrators of the queue to which this job is submitted to 
     can do all the modification operations.
     can do all the modification operations.
 
 
-    By default, nobody else besides job-owner, superuser, members of supergroup
-    and queue administrators can perform modification operations on a job.
+    By default, nobody else besides job-owner, the user who started the cluster,
+    cluster administrators and queue administrators can perform modification
+    operations on a job.
   </description>
   </description>
 </property>
 </property>
 
 
@@ -974,11 +976,11 @@
     user, for e.g., JobStatus, JobProfile, list of jobs in the queue, etc.
     user, for e.g., JobStatus, JobProfile, list of jobs in the queue, etc.
     
     
     Irrespective of this ACL configuration, job-owner, the user who started the
     Irrespective of this ACL configuration, job-owner, the user who started the
-    cluster, members of supergroup configured on JobTracker via
-    mapred.permissions.supergroup can do all the view operations.
+    cluster, cluster administrators configured on JobTracker via
+    mapreduce.cluster.administrators can do all the view operations.
     
     
-    By default, nobody else besides job-owner, superuser, members of supergroup
-    can perform view operations on a job.
+    By default, nobody else besides job-owner, the user who started the
+    cluster and cluster administrators can perform view operations on a job.
   </description>
   </description>
 </property>
 </property>
 
 

+ 16 - 22
src/mapred/org/apache/hadoop/mapred/ACLsManager.java

@@ -38,8 +38,7 @@ class ACLsManager {
 
 
   // MROwner(user who started this mapreduce cluster)'s ugi
   // MROwner(user who started this mapreduce cluster)'s ugi
   private final UserGroupInformation mrOwner;
   private final UserGroupInformation mrOwner;
-  // members of supergroup are mapreduce cluster administrators
-  private final String superGroup;
+  private final AccessControlList adminAcl;
   
   
   private final JobACLsManager jobACLsManager;
   private final JobACLsManager jobACLsManager;
   private final QueueManager queueManager;
   private final QueueManager queueManager;
@@ -55,10 +54,11 @@ class ACLsManager {
       mrOwner = UserGroupInformation.getCurrentUser();
       mrOwner = UserGroupInformation.getCurrentUser();
     }
     }
 
 
-    superGroup = conf.get(JobConf.MR_SUPERGROUP, "supergroup");
-    
     aclsEnabled = conf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
     aclsEnabled = conf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
 
 
+    adminAcl = new AccessControlList(conf.get(JobConf.MR_ADMINS, " "));
+    adminAcl.addUser(mrOwner.getShortUserName());
+
     this.jobACLsManager = jobACLsManager;
     this.jobACLsManager = jobACLsManager;
 
 
     this.queueManager = queueManager;
     this.queueManager = queueManager;
@@ -68,8 +68,8 @@ class ACLsManager {
     return mrOwner;
     return mrOwner;
   }
   }
 
 
-  String getSuperGroup() {
-    return superGroup;
+  AccessControlList getAdminsAcl() {
+    return adminAcl;
   }
   }
 
 
   JobACLsManager getJobACLsManager() {
   JobACLsManager getJobACLsManager() {
@@ -78,19 +78,13 @@ class ACLsManager {
 
 
   /**
   /**
    * Is the calling user an admin for the mapreduce cluster ?
    * Is the calling user an admin for the mapreduce cluster ?
-   * i.e. either cluster owner or member of mapred.permissions.supergroup.
+   * i.e. either cluster owner or member of mapreduce.cluster.administrators
    * @return true, if user is an admin
    * @return true, if user is an admin
    */
    */
   boolean isMRAdmin(UserGroupInformation callerUGI) {
   boolean isMRAdmin(UserGroupInformation callerUGI) {
-    if (mrOwner.getShortUserName().equals(callerUGI.getShortUserName())) {
+    if (adminAcl.isUserAllowed(callerUGI)) {
       return true;
       return true;
     }
     }
-    String[] groups = callerUGI.getGroupNames();
-    for(int i=0; i < groups.length; ++i) {
-      if (groups[i].equals(superGroup)) {
-        return true;
-      }
-    }
     return false;
     return false;
   }
   }
 
 
@@ -100,10 +94,10 @@ class ACLsManager {
    * <ul>
    * <ul>
    * <li>If ACLs are disabled, allow all users.</li>
    * <li>If ACLs are disabled, allow all users.</li>
    * <li>If the operation is not a job operation(for eg. submit-job-to-queue),
    * <li>If the operation is not a job operation(for eg. submit-job-to-queue),
-   *  then allow only (a) clusterOwner(who started the cluster), (b) members of
-   *  supergroup and (c) members of queue admins acl for the queue.</li>
+   *  then allow only (a) clusterOwner(who started the cluster), (b) cluster 
+   *  administrators (c) members of queue admins acl for the queue.</li>
    * <li>If the operation is a job operation, then allow only (a) jobOwner,
    * <li>If the operation is a job operation, then allow only (a) jobOwner,
-   * (b) clusterOwner(who started the cluster), (c) members of supergroup,
+   * (b) clusterOwner(who started the cluster), (c) cluster administrators,
    * (d) members of queue admins acl for the queue and (e) members of job
    * (d) members of queue admins acl for the queue and (e) members of job
    * acl for the jobOperation</li>
    * acl for the jobOperation</li>
    * </ul>
    * </ul>
@@ -134,7 +128,7 @@ class ACLsManager {
    * <ul>
    * <ul>
    * <li>If ACLs are disabled, allow all users.</li>
    * <li>If ACLs are disabled, allow all users.</li>
    * <li>Otherwise, allow only (a) jobOwner,
    * <li>Otherwise, allow only (a) jobOwner,
-   * (b) clusterOwner(who started the cluster), (c) members of supergroup,
+   * (b) clusterOwner(who started the cluster), (c) cluster administrators,
    * (d) members of job acl for the jobOperation</li>
    * (d) members of job acl for the jobOperation</li>
    * </ul>
    * </ul>
    */
    */
@@ -154,7 +148,7 @@ class ACLsManager {
    * <ul>
    * <ul>
    * <li>If ACLs are disabled, allow all users.</li>
    * <li>If ACLs are disabled, allow all users.</li>
    * <li>Otherwise, allow only (a) jobOwner,
    * <li>Otherwise, allow only (a) jobOwner,
-   * (b) clusterOwner(who started the cluster), (c) members of supergroup,
+   * (b) clusterOwner(who started the cluster), (c) cluster administrators,
    * (d) members of job acl for the jobOperation</li>
    * (d) members of job acl for the jobOperation</li>
    * </ul>
    * </ul>
    */
    */
@@ -171,10 +165,10 @@ class ACLsManager {
    * <ul>
    * <ul>
    * <li>If ACLs are disabled, allow all users.</li>
    * <li>If ACLs are disabled, allow all users.</li>
    * <li>If the operation is not a job operation(for eg. submit-job-to-queue),
    * <li>If the operation is not a job operation(for eg. submit-job-to-queue),
-   *  then allow only (a) clusterOwner(who started the cluster), (b) members of
-   *  supergroup and (c) members of queue admins acl for the queue.</li>
+   *  then allow only (a) clusterOwner(who started the cluster), (b)cluster 
+   *  administrators and (c) members of queue admins acl for the queue.</li>
    * <li>If the operation is a job operation, then allow only (a) jobOwner,
    * <li>If the operation is a job operation, then allow only (a) jobOwner,
-   * (b) clusterOwner(who started the cluster), (c) members of supergroup,
+   * (b) clusterOwner(who started the cluster), (c) cluster administrators,
    * (d) members of queue admins acl for the queue and (e) members of job
    * (d) members of queue admins acl for the queue and (e) members of job
    * acl for the jobOperation</li>
    * acl for the jobOperation</li>
    * </ul>
    * </ul>

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/JobACLsManager.java

@@ -61,7 +61,7 @@ class JobACLsManager {
 	      String aclConfigured = conf.get(aclConfigName);
 	      String aclConfigured = conf.get(aclConfigName);
 	      if (aclConfigured == null) {
 	      if (aclConfigured == null) {
 	        // If ACLs are not configured at all, we grant no access to anyone. So
 	        // If ACLs are not configured at all, we grant no access to anyone. So
-	        // jobOwner and superuser/supergroup _only_ can do 'stuff'
+	        // jobOwner and cluster administrators _only_ can do 'stuff'
 	        aclConfigured = "";
 	        aclConfigured = "";
 	      }
 	      }
 	      acls.put(aclName, new AccessControlList(aclConfigured));
 	      acls.put(aclName, new AccessControlList(aclConfigured));

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/JobConf.java

@@ -166,7 +166,7 @@ public class JobConf extends Configuration {
 
 
   static final String MR_ACLS_ENABLED = "mapred.acls.enabled";
   static final String MR_ACLS_ENABLED = "mapred.acls.enabled";
 
 
-  static final String MR_SUPERGROUP = "mapred.permissions.supergroup";
+  static final String MR_ADMINS = "mapreduce.cluster.administrators";
 
 
   /**
   /**
    * Configuration key to set the java command line options for the child
    * Configuration key to set the java command line options for the child

+ 4 - 13
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -2086,8 +2086,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     aclsManager = new ACLsManager(conf, new JobACLsManager(conf), queueManager);
     aclsManager = new ACLsManager(conf, new JobACLsManager(conf), queueManager);
 
 
     LOG.info("Starting jobtracker with owner as " +
     LOG.info("Starting jobtracker with owner as " +
-        getMROwner().getShortUserName() + " and supergroup as " +
-        getSuperGroup());
+        getMROwner().getShortUserName());
 
 
     // Create the scheduler
     // Create the scheduler
     Class<? extends TaskScheduler> schedulerClass
     Class<? extends TaskScheduler> schedulerClass
@@ -2123,7 +2122,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     int tmpInfoPort = infoSocAddr.getPort();
     int tmpInfoPort = infoSocAddr.getPort();
     this.startTime = clock.getTime();
     this.startTime = clock.getTime();
     infoServer = new HttpServer("job", infoBindAddress, tmpInfoPort, 
     infoServer = new HttpServer("job", infoBindAddress, tmpInfoPort, 
-        tmpInfoPort == 0, conf);
+        tmpInfoPort == 0, conf, aclsManager.getAdminsAcl());
     infoServer.setAttribute("job.tracker", this);
     infoServer.setAttribute("job.tracker", this);
     // initialize history parameters.
     // initialize history parameters.
     final JobTracker jtFinal = this;
     final JobTracker jtFinal = this;
@@ -4540,9 +4539,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   public synchronized void refreshNodes() throws IOException {
   public synchronized void refreshNodes() throws IOException {
     String user = UserGroupInformation.getCurrentUser().getShortUserName();
     String user = UserGroupInformation.getCurrentUser().getShortUserName();
     // check access
     // check access
-    if (!isMRAdmin(UserGroupInformation.getCurrentUser())) {
+    if (!aclsManager.isMRAdmin(UserGroupInformation.getCurrentUser())) {
       AuditLogger.logFailure(user, Constants.REFRESH_NODES, 
       AuditLogger.logFailure(user, Constants.REFRESH_NODES, 
-          getMROwner() + " " + getSuperGroup(), Constants.JOBTRACKER, 
+          aclsManager.getAdminsAcl().toString(), Constants.JOBTRACKER, 
           Constants.UNAUTHORIZED_USER);
           Constants.UNAUTHORIZED_USER);
       throw new AccessControlException(user + 
       throw new AccessControlException(user + 
                                        " is not authorized to refresh nodes.");
                                        " is not authorized to refresh nodes.");
@@ -4557,14 +4556,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     return aclsManager.getMROwner();
     return aclsManager.getMROwner();
   }
   }
 
 
-  String getSuperGroup() {
-    return aclsManager.getSuperGroup();
-  }
-
-  boolean isMRAdmin(UserGroupInformation ugi) {
-    return aclsManager.isMRAdmin(ugi);
-  }
-
   private synchronized void refreshHosts() throws IOException {
   private synchronized void refreshHosts() throws IOException {
     // Reread the config to get mapred.hosts and mapred.hosts.exclude filenames.
     // Reread the config to get mapred.hosts and mapred.hosts.exclude filenames.
     // Update the file names and refresh internal includes and excludes list
     // Update the file names and refresh internal includes and excludes list

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/TaskLogServlet.java

@@ -113,7 +113,7 @@ public class TaskLogServlet extends HttpServlet {
   /**
   /**
    * Validates if the given user has job view permissions for this job.
    * Validates if the given user has job view permissions for this job.
    * conf contains jobOwner and job-view-ACLs.
    * conf contains jobOwner and job-view-ACLs.
-   * We allow jobOwner, superUser(i.e. mrOwner) and members of superGroup and
+   * We allow jobOwner, superUser(i.e. mrOwner) and cluster administrators and
    * users and groups specified in configuration using
    * users and groups specified in configuration using
    * mapreduce.job.acl-view-job to view job.
    * mapreduce.job.acl-view-job to view job.
    */
    */

+ 9 - 13
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -568,6 +568,11 @@ public class TaskTracker
                             protocol);
                             protocol);
     }
     }
   }
   }
+  
+  int getHttpPort() {
+    return httpPort;
+  }
+
   public static final String TT_USER_NAME = "mapreduce.tasktracker.kerberos.principal";
   public static final String TT_USER_NAME = "mapreduce.tasktracker.kerberos.principal";
   public static final String TT_KEYTAB_FILE =
   public static final String TT_KEYTAB_FILE =
     "mapreduce.tasktracker.keytab.file";  
     "mapreduce.tasktracker.keytab.file";  
@@ -581,10 +586,8 @@ public class TaskTracker
     UserGroupInformation.setConfiguration(fConf);
     UserGroupInformation.setConfiguration(fConf);
     SecurityUtil.login(fConf, TT_KEYTAB_FILE, TT_USER_NAME);
     SecurityUtil.login(fConf, TT_KEYTAB_FILE, TT_USER_NAME);
 
 
-    aclsManager = new ACLsManager(fConf, new JobACLsManager(fConf), null);
-    LOG.info("Starting tasktracker with owner as " +
-        getMROwner().getShortUserName() + " and supergroup as " +
-        getSuperGroup());
+    LOG.info("Starting tasktracker with owner as "
+        + getMROwner().getShortUserName());
 
 
     localFs = FileSystem.getLocal(fConf);
     localFs = FileSystem.getLocal(fConf);
     if (fConf.get("slave.host.name") != null) {
     if (fConf.get("slave.host.name") != null) {
@@ -725,14 +728,6 @@ public class TaskTracker
     return aclsManager.getMROwner();
     return aclsManager.getMROwner();
   }
   }
 
 
-  String getSuperGroup() {
-    return aclsManager.getSuperGroup();
-  }
-
-  boolean isMRAdmin(UserGroupInformation ugi) {
-    return aclsManager.isMRAdmin(ugi);
-  }
-
   /**
   /**
    * Are ACLs for authorization checks enabled on the TT ?
    * Are ACLs for authorization checks enabled on the TT ?
    */
    */
@@ -1220,6 +1215,7 @@ public class TaskTracker
                   "mapred.tasktracker.map.tasks.maximum", 2);
                   "mapred.tasktracker.map.tasks.maximum", 2);
     maxReduceSlots = conf.getInt(
     maxReduceSlots = conf.getInt(
                   "mapred.tasktracker.reduce.tasks.maximum", 2);
                   "mapred.tasktracker.reduce.tasks.maximum", 2);
+    aclsManager = new ACLsManager(conf, new JobACLsManager(conf), null);
     this.jobTrackAddr = JobTracker.getAddress(conf);
     this.jobTrackAddr = JobTracker.getAddress(conf);
     String infoAddr = 
     String infoAddr = 
       NetUtils.getServerAddress(conf,
       NetUtils.getServerAddress(conf,
@@ -1230,7 +1226,7 @@ public class TaskTracker
     String httpBindAddress = infoSocAddr.getHostName();
     String httpBindAddress = infoSocAddr.getHostName();
     int httpPort = infoSocAddr.getPort();
     int httpPort = infoSocAddr.getPort();
     this.server = new HttpServer("task", httpBindAddress, httpPort,
     this.server = new HttpServer("task", httpBindAddress, httpPort,
-        httpPort == 0, conf);
+        httpPort == 0, conf, aclsManager.getAdminsAcl());
     workerThreads = conf.getInt("tasktracker.http.threads", 40);
     workerThreads = conf.getInt("tasktracker.http.threads", 40);
     this.shuffleServerMetrics = new ShuffleServerMetrics(conf);
     this.shuffleServerMetrics = new ShuffleServerMetrics(conf);
     server.setThreads(1, workerThreads);
     server.setThreads(1, workerThreads);

+ 3 - 4
src/test/org/apache/hadoop/http/TestHttpServer.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -276,9 +277,6 @@ public class TestHttpServer {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
         true);
         true);
-    conf.set(
-        CommonConfigurationKeys.HADOOP_CLUSTER_ADMINISTRATORS_PROPERTY,
-        "userA,userB groupC,groupD");
     conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
     conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
         DummyFilterInitializer.class.getName());
         DummyFilterInitializer.class.getName());
 
 
@@ -292,7 +290,8 @@ public class TestHttpServer {
     MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
     MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
     MyGroupsProvider.mapping.put("userE", Arrays.asList("groupE"));
     MyGroupsProvider.mapping.put("userE", Arrays.asList("groupE"));
 
 
-    HttpServer myServer = new HttpServer("test", "0.0.0.0", 0, true, conf);
+    HttpServer myServer = new HttpServer("test", "0.0.0.0", 0, true, conf,
+        new AccessControlList("userA,userB groupC,groupD"));
     myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.start();
     myServer.start();
     int port = myServer.getPort();
     int port = myServer.getPort();

+ 23 - 11
src/test/org/apache/hadoop/mapred/TestNodeRefresh.java

@@ -53,7 +53,7 @@ public class TestNodeRefresh extends TestCase {
   private JobTracker jt = null;
   private JobTracker jt = null;
   private String[] hosts = null;
   private String[] hosts = null;
   private String[] trackerHosts = null;
   private String[] trackerHosts = null;
-  private UserGroupInformation owner, user1, user2, user3, user4;
+  private UserGroupInformation owner, user1, user2, user3, user4, user5;
   private static final Log LOG = 
   private static final Log LOG = 
     LogFactory.getLog(TestNodeRefresh.class);
     LogFactory.getLog(TestNodeRefresh.class);
   
   
@@ -76,6 +76,8 @@ public class TestNodeRefresh extends TestCase {
                                                        new String[] {"abc"});
                                                        new String[] {"abc"});
       user4= UserGroupInformation.createUserForTesting("user4", 
       user4= UserGroupInformation.createUserForTesting("user4", 
                                                    new String[] {"supergroup"});
                                                    new String[] {"supergroup"});
+      user5= UserGroupInformation.createUserForTesting("user5", 
+          new String[] {"user5"});
       conf.setBoolean("dfs.replication.considerLoad", false);
       conf.setBoolean("dfs.replication.considerLoad", false);
       
       
       // prepare hosts info
       // prepare hosts info
@@ -146,7 +148,7 @@ public class TestNodeRefresh extends TestCase {
 
 
   /**
   /**
    * Check default value of mapred.hosts.exclude. Also check if only 
    * Check default value of mapred.hosts.exclude. Also check if only 
-   * owner/supergroup user is allowed to this command.
+   * owner is allowed to this command.
    */
    */
   public void testMRRefreshDefault() throws IOException {  
   public void testMRRefreshDefault() throws IOException {  
     // start a cluster with 2 hosts and no exclude-hosts file
     // start a cluster with 2 hosts and no exclude-hosts file
@@ -176,14 +178,14 @@ public class TestNodeRefresh extends TestCase {
     assertTrue("Privileged user denied permission for refresh operation",
     assertTrue("Privileged user denied permission for refresh operation",
                success);
                success);
 
 
-    // refresh with super user
+    // refresh with supergroup
     success = false;
     success = false;
     client = getClient(conf, user4);
     client = getClient(conf, user4);
     try {
     try {
       client.refreshNodes();
       client.refreshNodes();
       success = true;
       success = true;
     } catch (IOException ioe){}
     } catch (IOException ioe){}
-    assertTrue("Super user denied permission for refresh operation",
+    assertFalse("Invalid user performed privileged refresh operation",
                success);
                success);
 
 
     // check the cluster status and tracker size
     // check the cluster status and tracker size
@@ -204,13 +206,13 @@ public class TestNodeRefresh extends TestCase {
   }
   }
 
 
   /**
   /**
-   * Check refresh with a specific user is set in the conf along with supergroup
+   * Check refresh with a specific user/group is set in the conf
    */
    */
   public void testMRSuperUsers() throws IOException {  
   public void testMRSuperUsers() throws IOException {  
-    // start a cluster with 1 host and specified superuser and supergroup
+    // start a cluster with 1 host and specified cluster administrators
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
-    // set the supergroup
-    conf.set(JobConf.MR_SUPERGROUP, "abc");
+    // set the admin acl
+    conf.set(JobConf.MR_ADMINS, "user5 abc");
     startCluster(2, 1, 0, UserGroupInformation.createRemoteUser("user1"), conf);
     startCluster(2, 1, 0, UserGroupInformation.createRemoteUser("user1"), conf);
     
     
     conf = mr.createJobConf(new JobConf(conf));
     conf = mr.createJobConf(new JobConf(conf));
@@ -235,14 +237,24 @@ public class TestNodeRefresh extends TestCase {
     assertTrue("Privileged user denied permission for refresh operation",
     assertTrue("Privileged user denied permission for refresh operation",
                success);
                success);
 
 
-    // refresh with super user
+    // refresh with admin group
     success = false;
     success = false;
     client = getClient(conf, user3);
     client = getClient(conf, user3);
     try {
     try {
       client.refreshNodes();
       client.refreshNodes();
       success = true;
       success = true;
     } catch (IOException ioe){}
     } catch (IOException ioe){}
-    assertTrue("Super user denied permission for refresh operation",
+    assertTrue("Admin group member denied permission for refresh operation",
+               success);
+
+    // refresh with admin user
+    success = false;
+    client = getClient(conf, user5);
+    try {
+      client.refreshNodes();
+      success = true;
+    } catch (IOException ioe){}
+    assertTrue("Admin user denied permission for refresh operation",
                success);
                success);
 
 
     stopCluster();
     stopCluster();
@@ -250,7 +262,7 @@ public class TestNodeRefresh extends TestCase {
 
 
   /**
   /**
    * Check node refresh for decommissioning. Check if an allowed host is 
    * Check node refresh for decommissioning. Check if an allowed host is 
-   * disallowed upon refresh. Also check if only owner/supergroup user is 
+   * disallowed upon refresh. Also check if only owner/cluster administrator is 
    * allowed to fire this command.
    * allowed to fire this command.
    */
    */
   public void testMRRefreshDecommissioning() throws IOException {
   public void testMRRefreshDecommissioning() throws IOException {

+ 15 - 8
src/test/org/apache/hadoop/mapred/TestQueueManager.java

@@ -119,9 +119,12 @@ public class TestQueueManager extends TestCase {
     String groupName = "group1";
     String groupName = "group1";
     verifyJobSubmissionToDefaultQueue(conf, false, userName + "," + groupName);
     verifyJobSubmissionToDefaultQueue(conf, false, userName + "," + groupName);
     
     
-    // Check if member of supergroup can submit job
-    conf.set(JobConf.MR_SUPERGROUP, groupName);
+    // Check if admins can submit job
+    String user2 = "user2";
+    String group2 = "group2";
+    conf.set(JobConf.MR_ADMINS, user2 + " " + groupName);
     verifyJobSubmissionToDefaultQueue(conf, true, userName + "," + groupName);
     verifyJobSubmissionToDefaultQueue(conf, true, userName + "," + groupName);
+    verifyJobSubmissionToDefaultQueue(conf, true, user2 + "," + group2);
     
     
     // Check if MROwner(user who started the mapreduce cluster) can submit job
     // Check if MROwner(user who started the mapreduce cluster) can submit job
     UserGroupInformation mrOwner = UserGroupInformation.getCurrentUser();
     UserGroupInformation mrOwner = UserGroupInformation.getCurrentUser();
@@ -234,9 +237,11 @@ public class TestQueueManager extends TestCase {
     // Create a fake superuser for all processes to execute within
     // Create a fake superuser for all processes to execute within
     final UserGroupInformation ugi = createNecessaryUsers();
     final UserGroupInformation ugi = createNecessaryUsers();
 
 
-    // create other user who will try to kill the job of ugi.
-    final UserGroupInformation otherUGI = UserGroupInformation.
+    // create other users who will try to kill the job of ugi.
+    final UserGroupInformation otherUGI1 = UserGroupInformation.
         createUserForTesting("user1", new String [] {"group1"});
         createUserForTesting("user1", new String [] {"group1"});
+    final UserGroupInformation otherUGI2 = UserGroupInformation.
+    createUserForTesting("user2", new String [] {"group2"});
 
 
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
 
 
@@ -247,11 +252,13 @@ public class TestQueueManager extends TestCase {
             "default", adminAcl), " ");
             "default", adminAcl), " ");
         // Run job as ugi and try to kill job as user1, who (obviously)
         // Run job as ugi and try to kill job as user1, who (obviously)
         // should not be able to kill the job.
         // should not be able to kill the job.
-        verifyJobKill(otherUGI, conf, false);
+        verifyJobKill(otherUGI1, conf, false);
+        verifyJobKill(otherUGI2, conf, false);
 
 
-        // Check if member of supergroup can kill job
-        conf.set(JobConf.MR_SUPERGROUP, "group1");
-        verifyJobKill(otherUGI, conf, true);
+        // Check if cluster administrator can kill job
+        conf.set(JobConf.MR_ADMINS, "user1 group2");
+        verifyJobKill(otherUGI1, conf, true);
+        verifyJobKill(otherUGI2, conf, true);
         
         
         // Check if MROwner(user who started the mapreduce cluster) can kill job
         // Check if MROwner(user who started the mapreduce cluster) can kill job
         verifyJobKill(ugi, conf, true);
         verifyJobKill(ugi, conf, true);

+ 102 - 55
src/test/org/apache/hadoop/mapred/TestWebUIAuthorization.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.JobHistory.Keys;
 import org.apache.hadoop.mapred.JobHistory.TaskAttempt;
 import org.apache.hadoop.mapred.JobHistory.TaskAttempt;
 import org.apache.hadoop.mapred.QueueManager.QueueOperation;
 import org.apache.hadoop.mapred.QueueManager.QueueOperation;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.examples.SleepJob;
 import org.apache.hadoop.examples.SleepJob;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
@@ -61,10 +62,14 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
 
 
   // mrOwner starts the cluster
   // mrOwner starts the cluster
   private static String mrOwner = null;
   private static String mrOwner = null;
-  // member of supergroup
-  private static final String superGroupMember = "user2";
   // admin of "default" queue
   // admin of "default" queue
   private static final String qAdmin = "user3";
   private static final String qAdmin = "user3";
+  // mrAdmin
+  private static final String mrAdminUser = "user4";
+  // Group for mrAdmins
+  private static final String mrAdminGroup = "admingroup";
+  // member of mrAdminGroup
+  private static final String mrAdminGroupMember = "user2";
   // "colleague1" is there in job-view-acls config
   // "colleague1" is there in job-view-acls config
   private static final String viewColleague = "colleague1";
   private static final String viewColleague = "colleague1";
   // "colleague2" is there in job-modify-acls config
   // "colleague2" is there in job-modify-acls config
@@ -114,7 +119,7 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
    * Validates the given jsp/servlet against different user names who
    * Validates the given jsp/servlet against different user names who
    * can(or cannot) view the job.
    * can(or cannot) view the job.
    * (1) jobSubmitter can view the job
    * (1) jobSubmitter can view the job
-   * (2) superGroupMember can view any job
+   * (2) mrAdmin can view any job
    * (3) mrOwner can view any job
    * (3) mrOwner can view any job
    * (4) user mentioned in job-view-acls should be able to view the
    * (4) user mentioned in job-view-acls should be able to view the
    *     job irrespective of job-modify-acls.
    *     job irrespective of job-modify-acls.
@@ -127,9 +132,12 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     assertEquals("Incorrect return code for job submitter " + jobSubmitter,
     assertEquals("Incorrect return code for job submitter " + jobSubmitter,
         HttpURLConnection.HTTP_OK, getHttpStatusCode(url, jobSubmitter,
         HttpURLConnection.HTTP_OK, getHttpStatusCode(url, jobSubmitter,
             method));
             method));
-    assertEquals("Incorrect return code for supergroup-member " +
-        superGroupMember, HttpURLConnection.HTTP_OK,
-        getHttpStatusCode(url, superGroupMember, method));
+    assertEquals("Incorrect return code for admin user " +
+        mrAdminUser, HttpURLConnection.HTTP_OK,
+        getHttpStatusCode(url, mrAdminUser, method));
+    assertEquals("Incorrect return code for admingroup-member " +
+        mrAdminGroupMember, HttpURLConnection.HTTP_OK,
+        getHttpStatusCode(url, mrAdminGroupMember, method));
     assertEquals("Incorrect return code for MR-owner " + mrOwner,
     assertEquals("Incorrect return code for MR-owner " + mrOwner,
         HttpURLConnection.HTTP_OK, getHttpStatusCode(url, mrOwner, method));
         HttpURLConnection.HTTP_OK, getHttpStatusCode(url, mrOwner, method));
     assertEquals("Incorrect return code for user in job-view-acl " +
     assertEquals("Incorrect return code for user in job-view-acl " +
@@ -149,7 +157,7 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
   /**
   /**
    * Validates the given jsp/servlet against different user names who
    * Validates the given jsp/servlet against different user names who
    * can(or cannot) modify the job.
    * can(or cannot) modify the job.
-   * (1) jobSubmitter, mrOwner, qAdmin and superGroupMember can modify the job.
+   * (1) jobSubmitter, mrOwner, qAdmin and mrAdmin can modify the job.
    *     But we are not validating this in this method. Let the caller
    *     But we are not validating this in this method. Let the caller
    *     explicitly validate this, if needed.
    *     explicitly validate this, if needed.
    * (2) user mentioned in job-view-acls but not in job-modify-acls cannot
    * (2) user mentioned in job-view-acls but not in job-modify-acls cannot
@@ -257,11 +265,32 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     }
     }
   }
   }
 
 
-  public void testAuthorizationForJobHistoryPages() throws Exception {
-    JobConf conf = new JobConf();
+  static void setupGroupsProvider() throws IOException {
+    Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
     conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
         MyGroupsProvider.class.getName());
         MyGroupsProvider.class.getName());
     Groups.getUserToGroupsMappingService(conf);
     Groups.getUserToGroupsMappingService(conf);
+    MyGroupsProvider.mapping.put(jobSubmitter, Arrays.asList("group1"));
+    MyGroupsProvider.mapping.put(viewColleague, Arrays.asList("group2"));
+    MyGroupsProvider.mapping.put(modifyColleague, Arrays.asList("group1"));
+    MyGroupsProvider.mapping.put(unauthorizedUser, Arrays.asList("evilSociety"));
+    MyGroupsProvider.mapping.put(mrAdminGroupMember, Arrays.asList(mrAdminGroup));
+    MyGroupsProvider.mapping.put(viewAndModifyColleague, Arrays.asList("group3"));
+    MyGroupsProvider.mapping.put(qAdmin, Arrays.asList("group4"));
+
+    mrOwner = UserGroupInformation.getCurrentUser().getShortUserName();
+    MyGroupsProvider.mapping.put(mrOwner, Arrays.asList(
+        new String[] { "group5", "group6" }));
+    
+    MyGroupsProvider.mapping.put(jobSubmitter1, Arrays.asList("group7"));
+    MyGroupsProvider.mapping.put(jobSubmitter2, Arrays.asList("group7"));
+    MyGroupsProvider.mapping.put(jobSubmitter3, Arrays.asList("group7"));
+
+    MyGroupsProvider.mapping.put(mrAdminUser, Arrays.asList("group8"));
+  }
+
+  public void testAuthorizationForJobHistoryPages() throws Exception {
+    setupGroupsProvider();
     Properties props = new Properties();
     Properties props = new Properties();
     props.setProperty("hadoop.http.filter.initializers",
     props.setProperty("hadoop.http.filter.initializers",
         DummyFilterInitializer.class.getName());
         DummyFilterInitializer.class.getName());
@@ -278,25 +307,14 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     props.setProperty("mapred.job.tracker.history.completed.location",
     props.setProperty("mapred.job.tracker.history.completed.location",
         "historyDoneFolderOnHDFS");
         "historyDoneFolderOnHDFS");
 
 
-    props.setProperty(JobConf.MR_SUPERGROUP, "superGroup");
+    props.setProperty(JobConf.MR_ADMINS, mrAdminUser + " " + mrAdminGroup);
 
 
-    MyGroupsProvider.mapping.put(jobSubmitter, Arrays.asList("group1"));
-    MyGroupsProvider.mapping.put(viewColleague, Arrays.asList("group2"));
-    MyGroupsProvider.mapping.put(modifyColleague, Arrays.asList("group1"));
-    MyGroupsProvider.mapping.put(unauthorizedUser, Arrays.asList("evilSociety"));
-    MyGroupsProvider.mapping.put(superGroupMember, Arrays.asList("superGroup"));
-    MyGroupsProvider.mapping.put(viewAndModifyColleague, Arrays.asList("group3"));
-    MyGroupsProvider.mapping.put(qAdmin, Arrays.asList("group4"));
-
-    mrOwner = UserGroupInformation.getCurrentUser().getShortUserName();
-    MyGroupsProvider.mapping.put(mrOwner, Arrays.asList(
-        new String[] { "group5", "group6" }));
 
 
     startCluster(true, props);
     startCluster(true, props);
     MiniMRCluster cluster = getMRCluster();
     MiniMRCluster cluster = getMRCluster();
     int infoPort = cluster.getJobTrackerRunner().getJobTrackerInfoPort();
     int infoPort = cluster.getJobTrackerRunner().getJobTrackerInfoPort();
 
 
-    conf = new JobConf(cluster.createJobConf());
+    JobConf conf = new JobConf(cluster.createJobConf());
     conf.set(JobContext.JOB_ACL_VIEW_JOB, viewColleague + " group3");
     conf.set(JobContext.JOB_ACL_VIEW_JOB, viewColleague + " group3");
 
 
     // Let us add group1 and group3 to modify-job-acl. So modifyColleague and
     // Let us add group1 and group3 to modify-job-acl. So modifyColleague and
@@ -418,12 +436,12 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
   /**
   /**
    * Starts a sleep job and tries to kill the job using jobdetails.jsp as
    * Starts a sleep job and tries to kill the job using jobdetails.jsp as
    * (1) viewColleague (2) unauthorizedUser (3) modifyColleague
    * (1) viewColleague (2) unauthorizedUser (3) modifyColleague
-   * (4) viewAndModifyColleague (5) mrOwner (6) superGroupMember and
+   * (4) viewAndModifyColleague (5) mrOwner (6) mrAdmin and
    * (7) jobSubmitter
    * (7) jobSubmitter
    *
    *
    * Validates the given jsp/servlet against different user names who
    * Validates the given jsp/servlet against different user names who
    * can(or cannot) do both view and modify on the job.
    * can(or cannot) do both view and modify on the job.
-   * (1) jobSubmitter, mrOwner and superGroupMember can do both view and modify
+   * (1) jobSubmitter, mrOwner and mrAdmin can do both view and modify
    *     on the job. But we are not validating this in this method. Let the
    *     on the job. But we are not validating this in this method. Let the
    *     caller explicitly validate this, if needed.
    *     caller explicitly validate this, if needed.
    * (2) user mentioned in job-view-acls and job-modify-acls can do this
    * (2) user mentioned in job-view-acls and job-modify-acls can do this
@@ -478,14 +496,16 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
       }
       }
     }
     }
 
 
-    // check if jobSubmitter, mrOwner superGroupMember can do
+    // check if jobSubmitter, mrOwner and mrAdmin can do
     // killJob using jobdetails.jsp url
     // killJob using jobdetails.jsp url
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
                                        jobSubmitter);
                                        jobSubmitter);
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
                                        mrOwner);
                                        mrOwner);
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
-                                       superGroupMember);
+                                       mrAdminGroupMember);
+    confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
+        mrAdminUser);
   }
   }
 
 
   /**
   /**
@@ -555,7 +575,7 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     conf.set(JobContext.JOB_ACL_VIEW_JOB, " ");
     conf.set(JobContext.JOB_ACL_VIEW_JOB, " ");
     
     
     // Let us start 4 jobs as 4 different users(none of these 4 users is
     // Let us start 4 jobs as 4 different users(none of these 4 users is
-    // mrOwner and none of these users is a member of superGroup and none of
+    // mrOwner and none of these users is a member of mrAdmin and none of
     // these 4 users is a queue admin for the default queue). So only
     // these 4 users is a queue admin for the default queue). So only
     // based on the config JobContext.JOB_ACL_MODIFY_JOB being set here and the
     // based on the config JobContext.JOB_ACL_MODIFY_JOB being set here and the
     // job-submitter, killJob on each of the jobs will be succeeded.
     // job-submitter, killJob on each of the jobs will be succeeded.
@@ -620,10 +640,7 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
    */
    */
   @Test
   @Test
   public void testWebUIAuthorization() throws Exception {
   public void testWebUIAuthorization() throws Exception {
-    JobConf conf = new JobConf();
-    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
-        MyGroupsProvider.class.getName());
-    Groups.getUserToGroupsMappingService(conf);
+    setupGroupsProvider();
     Properties props = new Properties();
     Properties props = new Properties();
     props.setProperty("hadoop.http.filter.initializers",
     props.setProperty("hadoop.http.filter.initializers",
         DummyFilterInitializer.class.getName());
         DummyFilterInitializer.class.getName());
@@ -641,30 +658,13 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     props.setProperty("mapred.tasktracker.map.tasks.maximum", "6");
     props.setProperty("mapred.tasktracker.map.tasks.maximum", "6");
 
 
     props.setProperty(JSPUtil.PRIVATE_ACTIONS_KEY, "true");
     props.setProperty(JSPUtil.PRIVATE_ACTIONS_KEY, "true");
-    props.setProperty(JobConf.MR_SUPERGROUP, "superGroup");
-
-    MyGroupsProvider.mapping.put(jobSubmitter, Arrays.asList("group1"));
-    MyGroupsProvider.mapping.put(viewColleague, Arrays.asList("group2"));
-    MyGroupsProvider.mapping.put(modifyColleague, Arrays.asList("group1"));
-    MyGroupsProvider.mapping.put(unauthorizedUser, Arrays.asList("evilSociety"));
-    MyGroupsProvider.mapping.put(superGroupMember, Arrays.asList("superGroup"));
-    MyGroupsProvider.mapping.put(viewAndModifyColleague, Arrays.asList("group3"));
-    MyGroupsProvider.mapping.put(qAdmin, Arrays.asList("group4"));
-
-    mrOwner = UserGroupInformation.getCurrentUser().getShortUserName();
-    MyGroupsProvider.mapping.put(mrOwner, Arrays.asList(
-        new String[] { "group5", "group6" }));
-    
-    MyGroupsProvider.mapping.put(jobSubmitter1, Arrays.asList("group7"));
-    MyGroupsProvider.mapping.put(jobSubmitter2, Arrays.asList("group7"));
-    MyGroupsProvider.mapping.put(jobSubmitter3, Arrays.asList("group7"));
-
+    props.setProperty(JobConf.MR_ADMINS, mrAdminUser + " " + mrAdminGroup);
     startCluster(true, props);
     startCluster(true, props);
     MiniMRCluster cluster = getMRCluster();
     MiniMRCluster cluster = getMRCluster();
     int infoPort = cluster.getJobTrackerRunner().getJobTrackerInfoPort();
     int infoPort = cluster.getJobTrackerRunner().getJobTrackerInfoPort();
 
 
     JobConf clusterConf = cluster.createJobConf();
     JobConf clusterConf = cluster.createJobConf();
-    conf = new JobConf(clusterConf);
+    JobConf conf = new JobConf(clusterConf);
     conf.set(JobContext.JOB_ACL_VIEW_JOB, viewColleague + " group3");
     conf.set(JobContext.JOB_ACL_VIEW_JOB, viewColleague + " group3");
 
 
     // Let us add group1 and group3 to modify-job-acl. So modifyColleague and
     // Let us add group1 and group3 to modify-job-acl. So modifyColleague and
@@ -707,12 +707,13 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     validateJobDetailsJSPKillJob(cluster, clusterConf, jtURL);
     validateJobDetailsJSPKillJob(cluster, clusterConf, jtURL);
 
 
     // validate killJob of jobtracker.jsp as users viewAndModifyColleague,
     // validate killJob of jobtracker.jsp as users viewAndModifyColleague,
-    // jobSubmitter, mrOwner and superGroupMember
+    // jobSubmitter, mrOwner and mrAdmin
     confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL,
     confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL,
         viewAndModifyColleague);
         viewAndModifyColleague);
     confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, jobSubmitter);
     confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, jobSubmitter);
     confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, mrOwner);
     confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, mrOwner);
-    confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, superGroupMember);
+    confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, mrAdminUser);
+    confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, mrAdminGroupMember);
     confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, qAdmin);
     confirmJobTrackerJSPKillJobAsUser(cluster, conf, jtURL, qAdmin);
 
 
     // validate killing of multiple jobs using jobtracker jsp and check
     // validate killing of multiple jobs using jobtracker jsp and check
@@ -721,6 +722,48 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     validateKillMultipleJobs(cluster, conf, jtURL);
     validateKillMultipleJobs(cluster, conf, jtURL);
   }
   }
 
 
+  public void testWebUIAuthorizationForCommonServlets() throws Exception {
+    setupGroupsProvider();
+    Properties props = new Properties();
+    props.setProperty("hadoop.http.filter.initializers",
+        DummyFilterInitializer.class.getName());
+    props.setProperty(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, "true");
+    props.setProperty(JobConf.MR_ADMINS, mrAdminUser + " " + mrAdminGroup);
+
+    startCluster(true, props);
+    validateCommonServlets(getMRCluster());
+    stopCluster();
+  }
+
+  private void validateCommonServlets(MiniMRCluster cluster) throws IOException {
+    int infoPort = cluster.getJobTrackerRunner().getJobTrackerInfoPort();
+    String jtURL = "http://localhost:" + infoPort;
+    for (String servlet : new String[] { "logs", "stacks", "logLevel" }) {
+      String url = jtURL + "/" + servlet;
+      checkAccessToCommonServlet(url);
+    }
+    // validate access to common servlets for TaskTracker.
+    String ttURL = "http://localhost:"
+        + cluster.getTaskTrackerRunner(0).getTaskTracker().getHttpPort();
+    for (String servlet : new String[] { "logs", "stacks", "logLevel" }) {
+      String url = ttURL + "/" + servlet;
+      checkAccessToCommonServlet(url);
+    }
+  }
+
+  private void checkAccessToCommonServlet(String url) throws IOException {
+    url = url + "?a=b";
+    assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(url, mrAdminUser,
+        "GET"));
+    assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(url,
+        mrAdminGroupMember, "GET"));
+    assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(url,
+        mrOwner, "GET"));
+    // no access for any other user
+    assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, getHttpStatusCode(url,
+        jobSubmitter, "GET"));
+  }
+
   // validate killJob of jobtracker.jsp
   // validate killJob of jobtracker.jsp
   private void validateJobTrackerJSPKillJobAction(RunningJob job, String jtURL)
   private void validateJobTrackerJSPKillJobAction(RunningJob job, String jtURL)
       throws IOException {
       throws IOException {
@@ -761,13 +804,15 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
         "&changeJobPriority=true&setJobPriority="+"HIGH"+"&jobCheckBox=" +
         "&changeJobPriority=true&setJobPriority="+"HIGH"+"&jobCheckBox=" +
         jobid.toString();
         jobid.toString();
     validateModifyJob(jobTrackerJSPSetJobPriorityAction, "GET");
     validateModifyJob(jobTrackerJSPSetJobPriorityAction, "GET");
-    // jobSubmitter, mrOwner, qAdmin and superGroupMember are not validated for
+    // jobSubmitter, mrOwner, qAdmin and mrAdmin are not validated for
     // job-modify permission in validateModifyJob(). So let us do it
     // job-modify permission in validateModifyJob(). So let us do it
     // explicitly here
     // explicitly here
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
         jobTrackerJSPSetJobPriorityAction, jobSubmitter, "GET"));
         jobTrackerJSPSetJobPriorityAction, jobSubmitter, "GET"));
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
-        jobTrackerJSPSetJobPriorityAction, superGroupMember, "GET"));
+        jobTrackerJSPSetJobPriorityAction, mrAdminUser, "GET"));
+    assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
+        jobTrackerJSPSetJobPriorityAction, mrAdminGroupMember, "GET"));
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
         jobTrackerJSPSetJobPriorityAction, qAdmin, "GET"));
         jobTrackerJSPSetJobPriorityAction, qAdmin, "GET"));
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
@@ -848,6 +893,8 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     assertEquals(HttpURLConnection.HTTP_OK,
     assertEquals(HttpURLConnection.HTTP_OK,
         getHttpStatusCode(jobTrackerJSP, qAdmin, "GET"));
         getHttpStatusCode(jobTrackerJSP, qAdmin, "GET"));
     assertEquals(HttpURLConnection.HTTP_OK,
     assertEquals(HttpURLConnection.HTTP_OK,
-        getHttpStatusCode(jobTrackerJSP, superGroupMember, "GET"));
+        getHttpStatusCode(jobTrackerJSP, mrAdminUser, "GET"));
+    assertEquals(HttpURLConnection.HTTP_OK,
+        getHttpStatusCode(jobTrackerJSP, mrAdminGroupMember, "GET"));
   }
   }
 }
 }