Преглед изворни кода

commit 7486ae2d1d5b7d607dc1ec621e15fd08a971663f
Author: Vinod Kumar <vinodkv@yahoo-inc.com>
Date: Sat Apr 24 00:22:59 2010 +0530

MAPREDUCE:1664. From https://issues.apache.org/jira/secure/attachment/12442697/1664.20S.3.4.patch

+++ b/YAHOO-CHANGES.txt
+ MAPREDUCE-1664. Job Acls affect when Queue Acls are set.
+ (Ravi Gummadi via vinodkv)
+


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

Owen O'Malley пре 14 година
родитељ
комит
1783763770
26 измењених фајлова са 866 додато и 676 уклоњено
  1. 23 7
      conf/mapred-queue-acls.xml.template
  2. 13 18
      src/docs/src/documentation/content/xdocs/cluster_setup.xml
  3. 13 10
      src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
  4. 34 34
      src/mapred/mapred-default.xml
  5. 265 0
      src/mapred/org/apache/hadoop/mapred/ACLsManager.java
  6. 7 5
      src/mapred/org/apache/hadoop/mapred/CompletedJobStatusStore.java
  7. 12 6
      src/mapred/org/apache/hadoop/mapred/JSPUtil.java
  8. 21 55
      src/mapred/org/apache/hadoop/mapred/JobACLsManager.java
  9. 2 2
      src/mapred/org/apache/hadoop/mapred/JobConf.java
  10. 11 4
      src/mapred/org/apache/hadoop/mapred/JobHistory.java
  11. 0 22
      src/mapred/org/apache/hadoop/mapred/JobInProgress.java
  12. 55 112
      src/mapred/org/apache/hadoop/mapred/JobTracker.java
  13. 0 49
      src/mapred/org/apache/hadoop/mapred/JobTrackerJobACLsManager.java
  14. 10 60
      src/mapred/org/apache/hadoop/mapred/QueueManager.java
  15. 4 4
      src/mapred/org/apache/hadoop/mapred/TaskLogServlet.java
  16. 11 8
      src/mapred/org/apache/hadoop/mapred/TaskRunner.java
  17. 23 27
      src/mapred/org/apache/hadoop/mapred/TaskTracker.java
  18. 0 48
      src/mapred/org/apache/hadoop/mapred/TaskTrackerJobACLsManager.java
  19. 6 2
      src/test/org/apache/hadoop/mapred/TestJobACLs.java
  20. 6 2
      src/test/org/apache/hadoop/mapred/TestJobHistory.java
  21. 4 2
      src/test/org/apache/hadoop/mapred/TestJobTrackerRestart.java
  22. 27 24
      src/test/org/apache/hadoop/mapred/TestQueueAclsForCurrentUser.java
  23. 244 129
      src/test/org/apache/hadoop/mapred/TestQueueManager.java
  24. 5 3
      src/test/org/apache/hadoop/mapred/TestRecoveryManager.java
  25. 1 0
      src/test/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java
  26. 69 43
      src/test/org/apache/hadoop/mapred/TestWebUIAuthorization.java

+ 23 - 7
conf/mapred-queue-acls.xml.template

@@ -7,24 +7,40 @@
 
 
 <property>
 <property>
   <name>mapred.queue.default.acl-submit-job</name>
   <name>mapred.queue.default.acl-submit-job</name>
-  <value>*</value>
+  <value> </value>
   <description> Comma separated list of user and group names that are allowed
   <description> Comma separated list of user and group names that are allowed
     to submit jobs to the 'default' queue. The user list and the group list
     to submit jobs to the 'default' queue. The user list and the group list
-    are separated by a blank. For e.g. alice,bob group1,group2. 
+    are separated by a blank. For e.g. user1,user2 group1,group2. 
     If set to the special value '*', it means all users are allowed to 
     If set to the special value '*', it means all users are allowed to 
-    submit jobs. 
+    submit jobs. If set to ' '(i.e. space), no user will be allowed to submit
+    jobs.
+
+    It is only used if authorization is enabled in Map/Reduce by setting the
+    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.
   </description>
   </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>mapred.queue.default.acl-administer-jobs</name>
   <name>mapred.queue.default.acl-administer-jobs</name>
-  <value>*</value>
+  <value> </value>
   <description> Comma separated list of user and group names that are allowed
   <description> Comma separated list of user and group names that are allowed
-    to delete jobs or modify job's priority for jobs not owned by the current
-    user in the 'default' queue. The user list and the group list
-    are separated by a blank. For e.g. alice,bob group1,group2. 
+    to delete jobs or modify job's priority for all the jobs
+    in the 'default' queue. The user list and the group list
+    are separated by a blank. For e.g. user1,user2 group1,group2. 
     If set to the special value '*', it means all users are allowed to do 
     If set to the special value '*', it means all users are allowed to do 
+    this operation. If set to ' '(i.e. space), no user will be allowed to do
     this operation.
     this operation.
+
+    It is only used if authorization is enabled in Map/Reduce by setting the
+    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.
   </description>
   </description>
 </property>
 </property>
 
 

+ 13 - 18
src/docs/src/documentation/content/xdocs/cluster_setup.xml

@@ -294,27 +294,22 @@
         </tr>
         </tr>
         <tr>
         <tr>
           <td>mapred.acls.enabled</td>
           <td>mapred.acls.enabled</td>
-          <td>Boolean, specifying whether queue ACLs are supported for 
-              authorizing job submission and job administration in a 
-              queue</td>
-          <td>
-            If <em>true</em>, queue ACLs would be checked while submitting
-            and administering jobs. ACLs can be specified using the
-            configuration parameters of the form
-            <em>mapred.queue.queue-name.acl-name</em>, defined below.
+          <td>Boolean, specifying whether checks for queue ACLs and job ACLs
+            are to be done for authorizing users for doing queue operations and
+            job operations.
           </td>
           </td>
-        </tr>
-        <tr>
-          <td>mapreduce.cluster.job-authorization-enabled</td>
-          <td>Boolean, specifying whether job ACLs are supported for 
-              authorizing view and modification of a job</td>
           <td>
           <td>
-            If <em>true</em>, job ACLs would be checked while viewing or
-            modifying a job. More details are available at 
-            <a href ="mapred_tutorial.html#Job+Authorization">Job Authorization</a>. 
+            If <em>true</em>, queue ACLs are checked while submitting
+            and administering jobs and job ACLs are checked for authorizing
+            view and modification of jobs. Queue ACLs are specified using the
+            configuration parameters of the form
+            <em>mapred.queue.queue-name.acl-name</em>, defined below under
+            mapred-queue-acls.xml. Job ACLs are described at 
+            <a href ="mapred_tutorial.html#Job+Authorization">Job Authorization
+            </a>
           </td>
           </td>
         </tr>
         </tr>
-		  </table>
+        </table>
       
       
       <p><br/><code> conf/mapred-queue-acls.xml</code></p>
       <p><br/><code> conf/mapred-queue-acls.xml</code></p>
       
       
@@ -337,7 +332,7 @@
           </td>
           </td>
         </tr>
         </tr>
         <tr>
         <tr>
-          <td>mapred.queue.<em>queue-name</em>.acl-administer-job</td>
+          <td>mapred.queue.<em>queue-name</em>.acl-administer-jobs</td>
           <td>List of users and groups that can change the priority
           <td>List of users and groups that can change the priority
               or kill jobs that have been submitted to the
               or kill jobs that have been submitted to the
               specified <em>queue-name</em>.</td>
               specified <em>queue-name</em>.</td>

+ 13 - 10
src/docs/src/documentation/content/xdocs/mapred_tutorial.xml

@@ -1508,12 +1508,14 @@
         
         
         <section>
         <section>
           <title>Job Authorization</title>
           <title>Job Authorization</title>
-          <p>Job level authorization is enabled on the cluster, if the configuration
-          <code>mapreduce.cluster.job-authorization-enabled</code> is set to
-          true. When enabled, access control checks are done by the JobTracker
-          and the TaskTracker before allowing users to view
-          job details or to modify a job using MapReduce APIs,
-          CLI or web user interfaces.</p>
+          <p>Job level authorization and queue level authorization are enabled
+          on the cluster, if the configuration
+          <code>mapred.acls.enabled</code> is set to
+          true. When enabled, access control checks are done by (a) the
+          JobTracker before allowing users to submit jobs to queues and
+          administering these jobs and (b) by the JobTracker and the TaskTracker 
+          before allowing users to view job details or to modify a job using
+          MapReduce APIs, CLI or web user interfaces.</p>
           
           
           <p>A job submitter can specify access control lists for viewing or
           <p>A job submitter can specify access control lists for viewing or
           modifying a job via the configuration properties
           modifying a job via the configuration properties
@@ -1547,10 +1549,11 @@
             <li> killing/failing a task of a job </li>
             <li> killing/failing a task of a job </li>
             <li> setting the priority of a job </li>
             <li> setting the priority of a job </li>
           </ul>
           </ul>
-          <p>These operations are also protected by the queue level ACL,
-          "acl-administer-jobs", configured via mapred-queue-acls.xml. The caller
-          will be authorized against both queue level ACLs and job level ACLs,
-          depending on what is enabled.</p>
+          <p>These operations are also permitted by the queue level ACL,
+          "mapred.queue.queue-name.acl-administer-jobs", configured via
+          mapred-queue-acls.xml. The caller will be able to do the operation
+          if he/she is part of either queue admins ACL or job modification ACL.
+          </p>
           
           
           <p>The format of a job level ACL is the same as the format for a
           <p>The format of a job level ACL is the same as the format for a
           queue level ACL as defined in the
           queue level ACL as defined in the

+ 34 - 34
src/mapred/mapred-default.xml

@@ -900,8 +900,14 @@
 <property>
 <property>
   <name>mapred.acls.enabled</name>
   <name>mapred.acls.enabled</name>
   <value>false</value>
   <value>false</value>
-  <description> Specifies whether ACLs are enabled, and should be checked
-    for various operations.
+  <description> Specifies whether ACLs should be checked
+    for authorization of users for doing various queue and job level operations.
+    ACLs are disabled by default. If enabled, access control checks are made by
+    JobTracker and TaskTracker when requests are made by users for queue
+    operations like submit job to a queue and kill a job in the queue and job
+    operations like viewing the job-details (See mapreduce.job.acl-view-job)
+    or for modifying the job (See mapreduce.job.acl-modify-job) using
+    Map/Reduce APIs, RPCs or via the console and web user interfaces.
   </description>
   </description>
 </property>
 </property>
 
 
@@ -916,69 +922,63 @@
   </description>
   </description>
 </property>
 </property>
 
 
-<property>
-  <name>mapreduce.cluster.job-authorization-enabled</name>
-  <value>false</value>
-  <description> Boolean flag that specifies if job-level authorization checks
-  should be enabled on the jobs submitted to the cluster.  Job-level
-  authorization is enabled if this flag is set to true or disabled otherwise.
-  It is disabled by default. If enabled, access control checks are made by
-  JobTracker and TaskTracker when requests are made by users for viewing the
-  job-details (See mapreduce.job.acl-view-job) or for modifying the job
-  (See mapreduce.job.acl-modify-job) using Map/Reduce APIs, RPCs or via the
-  console and web user interfaces.
-  </description>
-</property>
-
 <property>
 <property>
   <name>mapreduce.job.acl-modify-job</name>
   <name>mapreduce.job.acl-modify-job</name>
-  <value></value>
+  <value> </value>
   <description> Job specific access-control list for 'modifying' the job. It
   <description> Job specific access-control list for 'modifying' the job. It
     is only used if authorization is enabled in Map/Reduce by setting the
     is only used if authorization is enabled in Map/Reduce by setting the
-    configuration property mapreduce.cluster.job-authorization-enabled to true.
+    configuration property mapred.acls.enabled to true.
     This specifies the list of users and/or groups who can do modification
     This specifies the list of users and/or groups who can do modification
     operations on the job. For specifying a list of users and groups the
     operations on the job. For specifying a list of users and groups the
     format to use is "user1,user2 group1,group". If set to '*', it allows all
     format to use is "user1,user2 group1,group". If set to '*', it allows all
-    users/groups to modify this job. If set to '', it allows none. This
-    configuration is used to guard all the modifications with respect to this
-    job and takes care of all the following operations:
+    users/groups to modify this job. If set to ' '(i.e. space), it allows
+    none. This configuration is used to guard all the modifications with respect
+    to this job and takes care of all the following operations:
       o killing this job
       o killing this job
       o killing a task of this job, failing a task of this job
       o killing a task of this job, failing a task of this job
       o setting the priority of this job
       o setting the priority of this job
     Each of these operations are also protected by the per-queue level ACL
     Each of these operations are also protected by the per-queue level ACL
     "acl-administer-jobs" configured via mapred-queues.xml. So a caller should
     "acl-administer-jobs" configured via mapred-queues.xml. So a caller should
-    have the authorization to satisfy both the queue-level ACL and 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 and members
-    of supergroup configured on JobTracker via mapred.permissions.supergroup,
+    Irrespective of this ACL configuration, job-owner, superuser, members
+    of supergroup configured on JobTracker via mapred.permissions.supergroup
+    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/supergroup can
-    perform modification operations on a job that they don't own.
+    By default, nobody else besides job-owner, superuser, members of supergroup
+    and queue administrators can perform modification operations on a job.
   </description>
   </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>mapreduce.job.acl-view-job</name>
   <name>mapreduce.job.acl-view-job</name>
-  <value></value>
+  <value> </value>
   <description> Job specific access-control list for 'viewing' the job. It is
   <description> Job specific access-control list for 'viewing' the job. It is
     only used if authorization is enabled in Map/Reduce by setting the
     only used if authorization is enabled in Map/Reduce by setting the
-    configuration property mapreduce.cluster.job-authorization-enabled to true.
+    configuration property mapred.acls.enabled to true.
     This specifies the list of users and/or groups who can view private details
     This specifies the list of users and/or groups who can view private details
     about the job. For specifying a list of users and groups the
     about the job. For specifying a list of users and groups the
     format to use is "user1,user2 group1,group". If set to '*', it allows all
     format to use is "user1,user2 group1,group". If set to '*', it allows all
-    users/groups to modify this job. If set to '', it allows none. This
-    configuration is used to guard some of the job-views and at present only
-    protects APIs that can return possibly sensitive information of the
-    job-owner like
+    users/groups to modify this job. If set to ' '(i.e. space), it allows
+    none. This configuration is used to guard some of the job-views and at
+    present only protects APIs that can return possibly sensitive information
+    of the job-owner like
       o job-level counters
       o job-level counters
       o task-level counters
       o task-level counters
       o tasks' diagnostic information
       o tasks' diagnostic information
       o task-logs displayed on the TaskTracker web-UI and
       o task-logs displayed on the TaskTracker web-UI and
       o job.xml showed by the JobTracker's web-UI
       o job.xml showed by the JobTracker's web-UI
-    Every other piece information of jobs is still accessible by any other
-    users, for e.g., JobStatus, JobProfile, list of jobs in the queue, etc.
+    Every other piece of information of jobs is still accessible by any other
+    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
+    cluster, members of supergroup configured on JobTracker via
+    mapred.permissions.supergroup can do all the view operations.
+    
+    By default, nobody else besides job-owner, superuser, members of supergroup
+    can perform view operations on a job.
   </description>
   </description>
 </property>
 </property>
 
 

+ 265 - 0
src/mapred/org/apache/hadoop/mapred/ACLsManager.java

@@ -0,0 +1,265 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.AuditLogger.Constants;
+import org.apache.hadoop.mapred.QueueManager.QueueOperation;
+import org.apache.hadoop.mapreduce.JobACL;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+
+/**
+ * Manages MapReduce cluster administrators and access checks for
+ * job level operations and queue level operations.
+ * Uses JobACLsManager for access checks of job level operations and
+ * QueueManager for queue operations.
+ */
+class ACLsManager {
+
+  // MROwner(user who started this mapreduce cluster)'s ugi
+  private final UserGroupInformation mrOwner;
+  // members of supergroup are mapreduce cluster administrators
+  private final String superGroup;
+  
+  private final JobACLsManager jobACLsManager;
+  private final QueueManager queueManager;
+  
+  private final boolean aclsEnabled;
+
+  ACLsManager(Configuration conf, JobACLsManager jobACLsManager,
+      QueueManager queueManager) throws IOException {
+
+    if (UserGroupInformation.isLoginKeytabBased()) {
+      mrOwner = UserGroupInformation.getLoginUser();
+    } else {
+      mrOwner = UserGroupInformation.getCurrentUser();
+    }
+
+    superGroup = conf.get(JobConf.MR_SUPERGROUP, "supergroup");
+    
+    aclsEnabled = conf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
+
+    this.jobACLsManager = jobACLsManager;
+
+    this.queueManager = queueManager;
+  }
+
+  UserGroupInformation getMROwner() {
+    return mrOwner;
+  }
+
+  String getSuperGroup() {
+    return superGroup;
+  }
+
+  JobACLsManager getJobACLsManager() {
+    return jobACLsManager;
+  }
+
+  /**
+   * Is the calling user an admin for the mapreduce cluster ?
+   * i.e. either cluster owner or member of mapred.permissions.supergroup.
+   * @return true, if user is an admin
+   */
+  boolean isMRAdmin(UserGroupInformation callerUGI) {
+    if (mrOwner.getShortUserName().equals(callerUGI.getShortUserName())) {
+      return true;
+    }
+    String[] groups = callerUGI.getGroupNames();
+    for(int i=0; i < groups.length; ++i) {
+      if (groups[i].equals(superGroup)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check the ACLs for a user doing the passed queue-operation and the passed
+   * job operation.
+   * <ul>
+   * <li>If ACLs are disabled, allow all users.</li>
+   * <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>
+   * <li>If the operation is a job operation, then allow only (a) jobOwner,
+   * (b) clusterOwner(who started the cluster), (c) members of supergroup,
+   * (d) members of queue admins acl for the queue and (e) members of job
+   * acl for the jobOperation</li>
+   * </ul>
+   * 
+   * @param job
+   * @param callerUGI
+   * @param oper
+   * @param jobOperation
+   * @throws AccessControlException
+   * @throws IOException
+   */
+  void checkAccess(JobInProgress job,
+      UserGroupInformation callerUGI, QueueOperation qOperation,
+      JobACL jobOperation) throws AccessControlException {
+
+    String queue = job.getProfile().getQueueName();
+    String jobId = job.getJobID().toString();
+    JobStatus jobStatus = job.getStatus();
+    String jobOwner = jobStatus.getUsername();
+    AccessControlList jobAcl = jobStatus.getJobACLs().get(jobOperation);
+
+    checkAccess(jobId, callerUGI, queue, qOperation,
+        jobOperation, jobOwner, jobAcl);
+  }
+
+  /**
+   * Check the ACLs for a user doing the passed job operation.
+   * <ul>
+   * <li>If ACLs are disabled, allow all users.</li>
+   * <li>Otherwise, allow only (a) jobOwner,
+   * (b) clusterOwner(who started the cluster), (c) members of supergroup,
+   * (d) members of job acl for the jobOperation</li>
+   * </ul>
+   */
+  void checkAccess(JobStatus jobStatus, UserGroupInformation callerUGI,
+      JobACL jobOperation) throws AccessControlException {
+
+    String jobId = jobStatus.getJobID().toString();
+    String jobOwner = jobStatus.getUsername();
+    AccessControlList jobAcl = jobStatus.getJobACLs().get(jobOperation);
+
+    // If acls are enabled, check if jobOwner, cluster admin or part of job ACL
+    checkAccess(jobId, callerUGI, jobOperation, jobOwner, jobAcl);
+  }
+
+  /**
+   * Check the ACLs for a user doing the passed job operation.
+   * <ul>
+   * <li>If ACLs are disabled, allow all users.</li>
+   * <li>Otherwise, allow only (a) jobOwner,
+   * (b) clusterOwner(who started the cluster), (c) members of supergroup,
+   * (d) members of job acl for the jobOperation</li>
+   * </ul>
+   */
+  void checkAccess(String jobId, UserGroupInformation callerUGI,
+      JobACL jobOperation, String jobOwner, AccessControlList jobAcl)
+      throws AccessControlException {
+    // TODO: Queue admins are to be allowed to do the job view operation.
+    checkAccess(jobId, callerUGI, null, null, jobOperation, jobOwner, jobAcl);
+  }
+
+  /**
+   * Check the ACLs for a user doing the passed queue-operation and the passed
+   * job operation.
+   * <ul>
+   * <li>If ACLs are disabled, allow all users.</li>
+   * <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>
+   * <li>If the operation is a job operation, then allow only (a) jobOwner,
+   * (b) clusterOwner(who started the cluster), (c) members of supergroup,
+   * (d) members of queue admins acl for the queue and (e) members of job
+   * acl for the jobOperation</li>
+   * </ul>
+   * 
+   * callerUGI user who is trying to perform the qOperation/jobOperation.
+   * jobAcl could be job-view-acl or job-modify-acl depending on jobOperation.
+   */
+  void checkAccess(String jobId, UserGroupInformation callerUGI,
+      String queue, QueueOperation qOperation,
+      JobACL jobOperation, String jobOwner, AccessControlList jobAcl)
+      throws AccessControlException {
+    if (!aclsEnabled) {
+      return;
+    }
+
+    String user = callerUGI.getShortUserName();
+
+    // Allow mapreduce cluster admins to do any queue operation and
+    // any job operation
+    if (isMRAdmin(callerUGI)) {
+      if (qOperation == QueueOperation.SUBMIT_JOB) {
+        AuditLogger.logSuccess(user, qOperation.name(), queue);
+      } else {
+        AuditLogger.logSuccess(user, jobOperation.name(), jobId);
+      }
+      return;
+    }
+
+    if (qOperation == QueueOperation.SUBMIT_JOB) {
+      // This is strictly queue operation(not a job operation) like
+      // submit-job-to-queue.
+      if (!queueManager.hasAccess(queue, qOperation, callerUGI)) {
+        AuditLogger.logFailure(user, qOperation.name(), null, queue,
+            Constants.UNAUTHORIZED_USER + ", job : " + jobId);
+
+        throw new AccessControlException("User "
+            + callerUGI.getShortUserName() + " cannot perform "
+            + "operation " + qOperation + " on queue " + queue
+            + ".\n Please run \"hadoop queue -showacls\" "
+            + "command to find the queues you have access to .");
+      } else {
+        AuditLogger.logSuccess(user, qOperation.name(), queue);
+        return;
+      }
+    }
+
+    if (jobOperation == JobACL.VIEW_JOB) {
+      // check if jobOwner or part of acl-view-job
+      if (jobACLsManager.checkAccess(callerUGI, jobOperation,
+          jobOwner, jobAcl)) {
+        AuditLogger.logSuccess(user, jobOperation.name(), jobId.toString());
+        return;
+      }
+      else {
+        AuditLogger.logFailure(user, jobOperation.name(), null,
+            jobId.toString(), Constants.UNAUTHORIZED_USER);
+        throw new AccessControlException("User "
+            + callerUGI.getShortUserName() + " cannot perform operation "
+            + jobOperation + " on " + jobId);
+      }
+    }
+
+    if (jobOperation == JobACL.MODIFY_JOB) {
+      // check if queueAdmin, jobOwner or part of acl-modify-job
+      if (queueManager.hasAccess(queue, qOperation, callerUGI)) {
+        AuditLogger.logSuccess(user, qOperation.name(), queue);
+        return;
+      } else if (jobACLsManager.checkAccess(callerUGI, jobOperation,
+                 jobOwner, jobAcl)) {
+        AuditLogger.logSuccess(user, jobOperation.name(), jobId);
+        return;
+      }
+      AuditLogger.logFailure(user, jobOperation.name(), null,
+          jobId.toString(), Constants.UNAUTHORIZED_USER + ", queue : "
+          + queue);
+
+      throw new AccessControlException("User "
+          + callerUGI.getShortUserName() + " cannot perform operation "
+          + jobOperation + " on " + jobId + " that is in the queue "
+          + queue);
+    }
+
+    throw new AccessControlException("Unsupported queue operation "
+        + qOperation + " on queue " + queue + ", job operation "
+        + jobOperation + " on job " + jobId);
+  }
+
+}

+ 7 - 5
src/mapred/org/apache/hadoop/mapred/CompletedJobStatusStore.java

@@ -49,7 +49,7 @@ class CompletedJobStatusStore implements Runnable {
   private FileSystem fs;
   private FileSystem fs;
   private static final String JOB_INFO_STORE_DIR = "/jobtracker/jobsInfo";
   private static final String JOB_INFO_STORE_DIR = "/jobtracker/jobsInfo";
 
 
-  private JobACLsManager jobACLsManager = null;
+  private ACLsManager aclsManager;
 
 
   public static final Log LOG =
   public static final Log LOG =
           LogFactory.getLog(CompletedJobStatusStore.class);
           LogFactory.getLog(CompletedJobStatusStore.class);
@@ -57,7 +57,8 @@ class CompletedJobStatusStore implements Runnable {
   private static long HOUR = 1000 * 60 * 60;
   private static long HOUR = 1000 * 60 * 60;
   private static long SLEEP_TIME = 1 * HOUR;
   private static long SLEEP_TIME = 1 * HOUR;
 
 
-  CompletedJobStatusStore(JobACLsManager aclsManager, Configuration conf)
+
+  CompletedJobStatusStore(Configuration conf, ACLsManager aclsManager)
       throws IOException {
       throws IOException {
     active =
     active =
       conf.getBoolean("mapred.job.tracker.persist.jobstatus.active", false);
       conf.getBoolean("mapred.job.tracker.persist.jobstatus.active", false);
@@ -87,7 +88,7 @@ class CompletedJobStatusStore implements Runnable {
         deleteJobStatusDirs();
         deleteJobStatusDirs();
       }
       }
 
 
-      this.jobACLsManager = aclsManager;
+      this.aclsManager = aclsManager;
 
 
       LOG.info("Completed job store activated/configured with retain-time : " 
       LOG.info("Completed job store activated/configured with retain-time : " 
                + retainTime + " , job-info-dir : " + jobInfoDir);
                + retainTime + " , job-info-dir : " + jobInfoDir);
@@ -285,7 +286,7 @@ class CompletedJobStatusStore implements Runnable {
   }
   }
 
 
   /**
   /**
-   * This method retrieves Counters information from DFS stored using
+   * This method retrieves Counters information from file stored using
    * store method.
    * store method.
    *
    *
    * @param jobId the jobId for which Counters is queried
    * @param jobId the jobId for which Counters is queried
@@ -299,7 +300,8 @@ class CompletedJobStatusStore implements Runnable {
         FSDataInputStream dataIn = getJobInfoFile(jobId);
         FSDataInputStream dataIn = getJobInfoFile(jobId);
         if (dataIn != null) {
         if (dataIn != null) {
           JobStatus jobStatus = readJobStatus(dataIn);
           JobStatus jobStatus = readJobStatus(dataIn);
-          jobACLsManager.checkAccess(jobStatus,
+          // authorize the user for job view access
+          aclsManager.checkAccess(jobStatus,
               UserGroupInformation.getCurrentUser(), JobACL.VIEW_JOB);
               UserGroupInformation.getCurrentUser(), JobACL.VIEW_JOB);
           readJobProfile(dataIn);
           readJobProfile(dataIn);
           counters = readCounters(dataIn);
           counters = readCounters(dataIn);

+ 12 - 6
src/mapred/org/apache/hadoop/mapred/JSPUtil.java

@@ -95,14 +95,14 @@ class JSPUtil {
    *         and decide if view should be allowed or not. Job will be null if
    *         and decide if view should be allowed or not. Job will be null if
    *         the job with given jobid doesnot exist at the JobTracker.
    *         the job with given jobid doesnot exist at the JobTracker.
    */
    */
-  public static JobWithViewAccessCheck checkAccessAndGetJob(JobTracker jt,
+  public static JobWithViewAccessCheck checkAccessAndGetJob(final JobTracker jt,
       JobID jobid, HttpServletRequest request, HttpServletResponse response)
       JobID jobid, HttpServletRequest request, HttpServletResponse response)
       throws ServletException, IOException {
       throws ServletException, IOException {
     final JobInProgress job = jt.getJob(jobid);
     final JobInProgress job = jt.getJob(jobid);
     JobWithViewAccessCheck myJob = new JobWithViewAccessCheck(job);
     JobWithViewAccessCheck myJob = new JobWithViewAccessCheck(job);
 
 
     String user = request.getRemoteUser();
     String user = request.getRemoteUser();
-    if (user != null && job != null && jt.isJobLevelAuthorizationEnabled()) {
+    if (user != null && job != null && jt.areACLsEnabled()) {
       final UserGroupInformation ugi =
       final UserGroupInformation ugi =
         UserGroupInformation.createRemoteUser(user);
         UserGroupInformation.createRemoteUser(user);
       try {
       try {
@@ -110,7 +110,7 @@ class JSPUtil {
           public Void run() throws IOException, ServletException {
           public Void run() throws IOException, ServletException {
 
 
             // checks job view permission
             // checks job view permission
-            job.checkAccess(ugi, JobACL.VIEW_JOB);
+            jt.getACLsManager().checkAccess(job, ugi, null, JobACL.VIEW_JOB);
             return null;
             return null;
           }
           }
         });
         });
@@ -486,9 +486,11 @@ class JSPUtil {
     } else {
     } else {
       currentUser = UserGroupInformation.createRemoteUser(user);
       currentUser = UserGroupInformation.createRemoteUser(user);
     }
     }
-    jobTracker.getJobACLsManager().checkAccess(JobID.forName(jobid),
-        currentUser, JobACL.VIEW_JOB,
+
+    // Authorize the user for view access of this job
+    jobTracker.getACLsManager().checkAccess(jobid, currentUser, JobACL.VIEW_JOB,
         jobInfo.get(Keys.USER), jobInfo.getJobACLs().get(JobACL.VIEW_JOB));
         jobInfo.get(Keys.USER), jobInfo.getJobACLs().get(JobACL.VIEW_JOB));
+
     return jobInfo;
     return jobInfo;
   }
   }
 
 
@@ -559,7 +561,7 @@ class JSPUtil {
   static void printJobACLs(JobTracker tracker,
   static void printJobACLs(JobTracker tracker,
       Map<JobACL, AccessControlList> jobAcls, JspWriter out)
       Map<JobACL, AccessControlList> jobAcls, JspWriter out)
       throws IOException {
       throws IOException {
-    if (tracker.isJobLevelAuthorizationEnabled()) {
+    if (tracker.areACLsEnabled()) {
       // Display job-view-acls and job-modify-acls configured for this job
       // Display job-view-acls and job-modify-acls configured for this job
       out.print("<b>Job-ACLs:</b><br>");
       out.print("<b>Job-ACLs:</b><br>");
       for (JobACL aclName : JobACL.values()) {
       for (JobACL aclName : JobACL.values()) {
@@ -572,6 +574,10 @@ class JSPUtil {
         }
         }
       }
       }
     }
     }
+    else {
+      out.print("<b>Job-ACLs: " + new AccessControlList("*").toString()
+          + "</b><br>");
+    }
   }
   }
 
 
   static boolean privateActionsAllowed(JobConf conf) {
   static boolean privateActionsAllowed(JobConf conf) {

+ 21 - 55
src/mapred/org/apache/hadoop/mapred/JobACLsManager.java

@@ -20,8 +20,6 @@ package org.apache.hadoop.mapred;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapred.AuditLogger;
 import org.apache.hadoop.mapred.AuditLogger;
 import org.apache.hadoop.mapred.AuditLogger.Constants;
 import org.apache.hadoop.mapred.AuditLogger.Constants;
@@ -29,14 +27,17 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 
 
-public abstract class JobACLsManager {
+class JobACLsManager {
 
 
-	  static final Log LOG = LogFactory.getLog(JobACLsManager.class);
+	  JobConf conf;
+	  
+	  public JobACLsManager(JobConf conf) {
+        this.conf = conf;
+      }
 
 
-	  protected abstract boolean isJobLevelAuthorizationEnabled();
-
-	  protected abstract boolean isSuperUserOrSuperGroup(
-	      UserGroupInformation callerUGI);
+	  boolean areACLsEnabled() {
+	    return conf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
+	  }
 
 
 	  /**
 	  /**
 	   * Construct the jobACLs from the configuration so that they can be kept in
 	   * Construct the jobACLs from the configuration so that they can be kept in
@@ -51,7 +52,7 @@ public abstract class JobACLsManager {
 	      new HashMap<JobACL, AccessControlList>();
 	      new HashMap<JobACL, AccessControlList>();
 
 
 	    // Don't construct anything if authorization is disabled.
 	    // Don't construct anything if authorization is disabled.
-	    if (!isJobLevelAuthorizationEnabled()) {
+	    if (!areACLsEnabled()) {
 	      return acls;
 	      return acls;
 	    }
 	    }
 
 
@@ -69,70 +70,35 @@ public abstract class JobACLsManager {
 	  }
 	  }
 
 
 	  /**
 	  /**
-	   * If authorization is enabled, checks whether the user (in the callerUGI) is
-	   * authorized to perform the operation specified by 'jobOperation' on the job.
-	   * <ul>
-	   * <li>The owner of the job can do any operation on the job</li>
-	   * <li>The superuser/supergroup is always permitted to do operations on any
-	   * job.</li>
-	   * <li>For all other users/groups job-acls are checked</li>
-	   * </ul>
-	   * 
-	   * @param jobStatus
-	   * @param callerUGI
-	   * @param jobOperation
-	   */
-	  void checkAccess(JobStatus jobStatus, UserGroupInformation callerUGI,
-	      JobACL jobOperation) throws AccessControlException {
-
-	    JobID jobId = jobStatus.getJobID();
-	    String jobOwner = jobStatus.getUsername();
-	    AccessControlList acl = jobStatus.getJobACLs().get(jobOperation);
-	    checkAccess(jobId, callerUGI, jobOperation, jobOwner, acl);
-	  }
-
-	  /**
-	   * If authorization is enabled, checks whether the user (in the callerUGI) is
-	   * authorized to perform the operation specified by 'jobOperation' on the job.
+	   * If authorization is enabled, checks whether the user (in the callerUGI)
+	   * is authorized to perform the operation specified by 'jobOperation' on
+	   * the job by checking if the user is jobOwner or part of job ACL for the
+	   * specific job operation.
 	   * <ul>
 	   * <ul>
 	   * <li>The owner of the job can do any operation on the job</li>
 	   * <li>The owner of the job can do any operation on the job</li>
-	   * <li>The superuser/supergroup is always permitted to do operations on any
-	   * job.</li>
 	   * <li>For all other users/groups job-acls are checked</li>
 	   * <li>For all other users/groups job-acls are checked</li>
 	   * </ul>
 	   * </ul>
-	   * @param jobId
 	   * @param callerUGI
 	   * @param callerUGI
 	   * @param jobOperation
 	   * @param jobOperation
 	   * @param jobOwner
 	   * @param jobOwner
 	   * @param jobACL
 	   * @param jobACL
 	   * @throws AccessControlException
 	   * @throws AccessControlException
 	   */
 	   */
-	  void checkAccess(JobID jobId, UserGroupInformation callerUGI,
+	  boolean checkAccess(UserGroupInformation callerUGI,
 	      JobACL jobOperation, String jobOwner, AccessControlList jobACL)
 	      JobACL jobOperation, String jobOwner, AccessControlList jobACL)
 	      throws AccessControlException {
 	      throws AccessControlException {
 
 
 	    String user = callerUGI.getShortUserName();
 	    String user = callerUGI.getShortUserName();
-	    if (!isJobLevelAuthorizationEnabled()) {
-	      return;
+	    if (!areACLsEnabled()) {
+	      return true;
 	    }
 	    }
 
 
-	    // Allow superusers/supergroups
-	    // Allow Job-owner as the job's owner is always part of all the ACLs
-	    if (callerUGI.getShortUserName().equals(jobOwner)
-	        || isSuperUserOrSuperGroup(callerUGI) 
+	    // Allow Job-owner for any operation on the job
+	    if (user.equals(jobOwner) 
 	        || jobACL.isUserAllowed(callerUGI)) {
 	        || jobACL.isUserAllowed(callerUGI)) {
-	      AuditLogger.logSuccess(user, jobOperation.name(),  jobId.toString());
-	      return;
+	      return true;
 	    }
 	    }
 
 
-	    // log this event to the audit log
-	    AuditLogger.logFailure(user, jobOperation.name(), jobACL.toString(), 
-	                           jobId.toString(), Constants.UNAUTHORIZED_USER);
-	    throw new AccessControlException(callerUGI
-	        + " is not authorized for performing the operation "
-	        + jobOperation.toString() + " on " + jobId + ". "
-	        + jobOperation.toString()
-	        + " Access control list configured for this job : "
-	        + jobACL.toString());
+	    return false;
 	  }
 	  }
 	}
 	}

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

@@ -164,8 +164,8 @@ public class JobConf extends Configuration {
   static final String MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY =
   static final String MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY =
       "mapred.job.reduce.memory.mb";
       "mapred.job.reduce.memory.mb";
 
 
-  public static final String JOB_LEVEL_AUTHORIZATION_ENABLING_FLAG = 
-	    "mapreduce.cluster.job-authorization-enabled";
+  static final String MR_ACLS_ENABLED = "mapred.acls.enabled";
+
   static final String MR_SUPERGROUP = "mapred.permissions.supergroup";
   static final String MR_SUPERGROUP = "mapred.permissions.supergroup";
 
 
   /**
   /**

+ 11 - 4
src/mapred/org/apache/hadoop/mapred/JobHistory.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.mapred;
 
 
 import java.io.BufferedReader;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.File;
-import java.io.FileFilter;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
@@ -55,7 +54,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
@@ -117,6 +115,7 @@ public class JobHistory {
   private static FileSystem DONEDIR_FS; // Done dir filesystem
   private static FileSystem DONEDIR_FS; // Done dir filesystem
   private static JobConf jtConf;
   private static JobConf jtConf;
   private static Path DONE = null; // folder for completed jobs
   private static Path DONE = null; // folder for completed jobs
+  private static boolean aclsEnabled = false;
   /**
   /**
    * A filter for conf files
    * A filter for conf files
    */  
    */  
@@ -346,6 +345,9 @@ public class JobHistory {
                      3 * 1024 * 1024);
                      3 * 1024 * 1024);
       jtConf = conf;
       jtConf = conf;
 
 
+      // queue and job level security is enabled on the mapreduce cluster or not
+      aclsEnabled = conf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
+
       // initialize the file manager
       // initialize the file manager
       fileManager = new JobHistoryFilesManager(conf, jobTracker);
       fileManager = new JobHistoryFilesManager(conf, jobTracker);
     } catch(IOException e) {
     } catch(IOException e) {
@@ -1252,14 +1254,19 @@ public class JobHistory {
           // Log the history meta info
           // Log the history meta info
           JobHistory.MetaInfoManager.logMetaInfo(writers);
           JobHistory.MetaInfoManager.logMetaInfo(writers);
 
 
+          String viewJobACL = "*";
+          String modifyJobACL = "*";
+          if (aclsEnabled) {
+            viewJobACL = jobConf.get(JobACL.VIEW_JOB.getAclName(), " ");
+            modifyJobACL = jobConf.get(JobACL.MODIFY_JOB.getAclName(), " ");
+          }
           //add to writer as well 
           //add to writer as well 
           JobHistory.log(writers, RecordTypes.Job, 
           JobHistory.log(writers, RecordTypes.Job, 
                          new Keys[]{Keys.JOBID, Keys.JOBNAME, Keys.USER, Keys.SUBMIT_TIME, Keys.JOBCONF, 
                          new Keys[]{Keys.JOBID, Keys.JOBNAME, Keys.USER, Keys.SUBMIT_TIME, Keys.JOBCONF, 
                                       Keys.VIEW_JOB, Keys.MODIFY_JOB }, 
                                       Keys.VIEW_JOB, Keys.MODIFY_JOB }, 
                          new String[]{jobId.toString(), jobName, user, 
                          new String[]{jobId.toString(), jobName, user, 
                                       String.valueOf(submitTime) , jobConfPath,
                                       String.valueOf(submitTime) , jobConfPath,
-                                      jobConf.get(JobACL.VIEW_JOB.getAclName(), ""),
-                                      jobConf.get(JobACL.MODIFY_JOB.getAclName(), "")}
+                                      viewJobACL, modifyJobACL}
                         ); 
                         ); 
              
              
         }catch(IOException e){
         }catch(IOException e){

+ 0 - 22
src/mapred/org/apache/hadoop/mapred/JobInProgress.java

@@ -36,7 +36,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -44,7 +43,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.CleanupQueue.PathDeletionContext;
 import org.apache.hadoop.mapred.CleanupQueue.PathDeletionContext;
 import org.apache.hadoop.mapred.AuditLogger;
 import org.apache.hadoop.mapred.AuditLogger;
 import org.apache.hadoop.mapred.JobHistory.Values;
 import org.apache.hadoop.mapred.JobHistory.Values;
-import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobSubmissionFiles;
 import org.apache.hadoop.mapreduce.JobSubmissionFiles;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TaskType;
@@ -62,7 +60,6 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -707,25 +704,6 @@ public class JobInProgress {
     return allTaskSplitMetaInfo;
     return allTaskSplitMetaInfo;
   }
   }
 
 
-  /**
-   * If authorization is enabled on the JobTracker, checks whether the user (in
-   * the callerUGI) is authorized to perform the operation specify by
-   * 'jobOperation' on the job.
-   * <ul>
-   * <li>The owner of the job can do any operation on the job</li>
-   * <li>The superuser/supergroup of the JobTracker is always permitted to do
-   * operations on any job.</li>
-   * <li>For all other users/groups job-acls are checked</li>
-   * </ul>
-   * 
-   * @param callerUGI
-   * @param jobOperation
-   */
-  void checkAccess(UserGroupInformation callerUGI, JobACL jobOperation)
-      throws AccessControlException {
-    jobtracker.getJobACLsManager().checkAccess(status, callerUGI, jobOperation);
-  }
-
   /////////////////////////////////////////////////////
   /////////////////////////////////////////////////////
   // Accessors for the JobInProgress
   // Accessors for the JobInProgress
   /////////////////////////////////////////////////////
   /////////////////////////////////////////////////////

+ 55 - 112
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -19,18 +19,14 @@ package org.apache.hadoop.mapred;
 
 
 
 
 import java.io.BufferedReader;
 import java.io.BufferedReader;
-import java.io.BufferedWriter;
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.io.Writer;
 import java.net.BindException;
 import java.net.BindException;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
@@ -74,7 +70,6 @@ import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenSecr
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
 import org.apache.hadoop.mapred.AuditLogger.Constants;
 import org.apache.hadoop.mapred.AuditLogger.Constants;
@@ -96,6 +91,7 @@ import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
@@ -1697,8 +1693,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
 
 
           // check the access
           // check the access
           try {
           try {
-            checkAccess(job, ugi, QueueManager.QueueOperation.SUBMIT_JOB,
-                        null);
+            aclsManager.checkAccess(job, ugi,
+                QueueManager.QueueOperation.SUBMIT_JOB, null);
           } catch (Throwable t) {
           } catch (Throwable t) {
             LOG.warn("Access denied for user " + ugi.getShortUserName() 
             LOG.warn("Access denied for user " + ugi.getShortUserName() 
                      + " in groups : [" 
                      + " in groups : [" 
@@ -1942,7 +1938,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
                                                 "expireLaunchingTasks");
                                                 "expireLaunchingTasks");
 
 
   CompletedJobStatusStore completedJobStatusStore = null;
   CompletedJobStatusStore completedJobStatusStore = null;
-  private JobTrackerJobACLsManager jobACLsManager;
   Thread completedJobsStoreThread = null;
   Thread completedJobsStoreThread = null;
   RecoveryManager recoveryManager;
   RecoveryManager recoveryManager;
 
 
@@ -1982,8 +1977,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   FileSystem fs = null;
   FileSystem fs = null;
   Path systemDir = null;
   Path systemDir = null;
   JobConf conf;
   JobConf conf;
-  private final UserGroupInformation mrOwner;
-  private final String supergroup;
+
+  private final ACLsManager aclsManager;
 
 
   long limitMaxMemForMapTasks;
   long limitMaxMemForMapTasks;
   long limitMaxMemForReduceTasks;
   long limitMaxMemForReduceTasks;
@@ -2024,16 +2019,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     // get the desired principal to load
     // get the desired principal to load
     UserGroupInformation.setConfiguration(conf);
     UserGroupInformation.setConfiguration(conf);
     SecurityUtil.login(conf, JT_KEYTAB_FILE, JT_USER_NAME, localMachine);
     SecurityUtil.login(conf, JT_KEYTAB_FILE, JT_USER_NAME, localMachine);
-    if (UserGroupInformation.isLoginKeytabBased()) {
-      mrOwner = UserGroupInformation.getLoginUser();
-    } else {
-      mrOwner = UserGroupInformation.getCurrentUser();
-    }
-  
-    supergroup = conf.get(JobConf.MR_SUPERGROUP,
-                          "supergroup");
-    LOG.info("Starting jobtracker with owner as " + mrOwner.getShortUserName() 
-             + " and supergroup as " + supergroup);
+
     long secretKeyInterval = 
     long secretKeyInterval = 
     conf.getLong(DELEGATION_KEY_UPDATE_INTERVAL_KEY, 
     conf.getLong(DELEGATION_KEY_UPDATE_INTERVAL_KEY, 
                    DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT);
                    DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT);
@@ -2096,7 +2082,13 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
 
 
     Configuration queuesConf = new Configuration(this.conf);
     Configuration queuesConf = new Configuration(this.conf);
     queueManager = new QueueManager(queuesConf);
     queueManager = new QueueManager(queuesConf);
-    
+
+    aclsManager = new ACLsManager(conf, new JobACLsManager(conf), queueManager);
+
+    LOG.info("Starting jobtracker with owner as " +
+        getMROwner().getShortUserName() + " and supergroup as " +
+        getSuperGroup());
+
     // Create the scheduler
     // Create the scheduler
     Class<? extends TaskScheduler> schedulerClass
     Class<? extends TaskScheduler> schedulerClass
       = conf.getClass("mapred.jobtracker.taskScheduler",
       = conf.getClass("mapred.jobtracker.taskScheduler",
@@ -2136,7 +2128,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     // initialize history parameters.
     // initialize history parameters.
     final JobTracker jtFinal = this;
     final JobTracker jtFinal = this;
     boolean historyInitialized = 
     boolean historyInitialized = 
-      mrOwner.doAs(new PrivilegedExceptionAction<Boolean>() {
+      getMROwner().doAs(new PrivilegedExceptionAction<Boolean>() {
         @Override
         @Override
         public Boolean run() throws Exception {
         public Boolean run() throws Exception {
           return JobHistory.init(jtFinal, conf,jtFinal.localMachine, 
           return JobHistory.init(jtFinal, conf,jtFinal.localMachine, 
@@ -2182,7 +2174,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       try {
       try {
         // if we haven't contacted the namenode go ahead and do it
         // if we haven't contacted the namenode go ahead and do it
         if (fs == null) {
         if (fs == null) {
-          fs = mrOwner.doAs(new PrivilegedExceptionAction<FileSystem>() {
+          fs = getMROwner().doAs(new PrivilegedExceptionAction<FileSystem>() {
             public FileSystem run() throws IOException {
             public FileSystem run() throws IOException {
               return FileSystem.get(conf);
               return FileSystem.get(conf);
           }});
           }});
@@ -2194,9 +2186,10 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
         }
         }
         try {
         try {
           FileStatus systemDirStatus = fs.getFileStatus(systemDir);
           FileStatus systemDirStatus = fs.getFileStatus(systemDir);
-          if (!systemDirStatus.getOwner().equals(mrOwner.getShortUserName())) {
+          if (!systemDirStatus.getOwner().equals(
+              getMROwner().getShortUserName())) {
             throw new AccessControlException("The systemdir " + systemDir +
             throw new AccessControlException("The systemdir " + systemDir +
-                " is not owned by " + mrOwner.getShortUserName());
+                " is not owned by " + getMROwner().getShortUserName());
           }
           }
           if (!systemDirStatus.getPermission().equals(SYSTEM_DIR_PERMISSION)) {
           if (!systemDirStatus.getPermission().equals(SYSTEM_DIR_PERMISSION)) {
             LOG.warn("Incorrect permissions on " + systemDir +
             LOG.warn("Incorrect permissions on " + systemDir +
@@ -2257,7 +2250,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
 
 
     // Initialize history DONE folder
     // Initialize history DONE folder
     if (historyInitialized) {
     if (historyInitialized) {
-      FileSystem historyFS = mrOwner.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      FileSystem historyFS = getMROwner().doAs(
+          new PrivilegedExceptionAction<FileSystem>() {
         public FileSystem run() throws IOException {
         public FileSystem run() throws IOException {
           JobHistory.initDone(conf, fs);
           JobHistory.initDone(conf, fs);
           final String historyLogDir = 
           final String historyLogDir = 
@@ -2276,10 +2270,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     this.numTaskCacheLevels = conf.getInt("mapred.task.cache.levels", 
     this.numTaskCacheLevels = conf.getInt("mapred.task.cache.levels", 
         NetworkTopology.DEFAULT_HOST_LEVEL);
         NetworkTopology.DEFAULT_HOST_LEVEL);
 
 
-    // Initialize the jobACLSManager
-    jobACLsManager = new JobTrackerJobACLsManager(this);
     //initializes the job status store
     //initializes the job status store
-    completedJobStatusStore = new CompletedJobStatusStore(jobACLsManager, conf);
+    completedJobStatusStore = new CompletedJobStatusStore(conf, aclsManager);
   }
   }
 
 
   private static SimpleDateFormat getDateFormat() {
   private static SimpleDateFormat getDateFormat() {
@@ -3684,7 +3676,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
 
 
       // check for access
       // check for access
       try {
       try {
-        checkAccess(job, ugi, QueueManager.QueueOperation.SUBMIT_JOB, null);
+        aclsManager.checkAccess(job, ugi,
+            QueueManager.QueueOperation.SUBMIT_JOB, null);
       } catch (IOException ioe) {
       } catch (IOException ioe) {
         LOG.warn("Access denied for user " + job.getJobConf().getUser()
         LOG.warn("Access denied for user " + job.getJobConf().getUser()
             + ". Ignoring job " + jobId, ioe);
             + ". Ignoring job " + jobId, ioe);
@@ -3723,7 +3716,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     try{
     try{
       final String user =
       final String user =
         UserGroupInformation.getCurrentUser().getShortUserName();
         UserGroupInformation.getCurrentUser().getShortUserName();
-      return mrOwner.doAs(new PrivilegedExceptionAction<String>() {
+      return getMROwner().doAs(new PrivilegedExceptionAction<String>() {
         @Override
         @Override
         public String run() throws Exception {
         public String run() throws Exception {
           return getStagingAreaDirInternal(user);
           return getStagingAreaDirInternal(user);
@@ -3774,54 +3767,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   }
   }
 
 
   /**
   /**
-   * Is job-level authorization enabled on the JT?
+   * Are ACLs for authorization checks enabled on the JT?
    * 
    * 
    * @return
    * @return
    */
    */
-  boolean isJobLevelAuthorizationEnabled() {
-    return conf.getBoolean(JobConf.JOB_LEVEL_AUTHORIZATION_ENABLING_FLAG, false);
-  }
-
-  /**
-   * Check the ACLs for a user doing the passed queue-operation and the passed
-   * job operation.
-   * <ul>
-   * <li>Superuser/supergroup can do any operation on the job</li>
-   * <li>For any other user/group, the configured ACLs for the corresponding
-   * queue and the job are checked.</li>
-   * </ul>
-   * 
-   * @param job
-   * @param callerUGI
-   * @param oper
-   * @param jobOperation
-   * @throws AccessControlException
-   * @throws IOException
-   */
-  private void checkAccess(JobInProgress job,
-      UserGroupInformation callerUGI, QueueManager.QueueOperation oper,
-      JobACL jobOperation) throws AccessControlException {
-
-    // get the queue and verify the queue access
-    String queue = job.getProfile().getQueueName();
-    if (!queueManager.hasAccess(queue, job, oper, callerUGI)) {
-      throw new AccessControlException("User " 
-                            + callerUGI.getShortUserName() 
-                            + " cannot perform "
-                            + "operation " + oper + " on queue " + queue +
-                            ".\n Please run \"hadoop queue -showacls\" " +
-                            "command to find the queues you have access" +
-                            " to .");
-    }
-
-    // check nulls, for e.g., submitJob RPC doesn't have a jobOperation as the
-    // job itself isn't created by that time.
-    if (jobOperation == null) {
-      return;
-    }
-
-    // check the access to the job
-    job.checkAccess(callerUGI, jobOperation);
+  boolean areACLsEnabled() {
+    return conf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
   }
   }
 
 
   /**@deprecated use {@link #getClusterStatus(boolean)}*/
   /**@deprecated use {@link #getClusterStatus(boolean)}*/
@@ -3884,7 +3835,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     }
     }
         
         
     // check both queue-level and job-level access
     // check both queue-level and job-level access
-    checkAccess(job, UserGroupInformation.getCurrentUser(),
+    aclsManager.checkAccess(job, UserGroupInformation.getCurrentUser(),
         QueueManager.QueueOperation.ADMINISTER_JOBS, JobACL.MODIFY_JOB);
         QueueManager.QueueOperation.ADMINISTER_JOBS, JobACL.MODIFY_JOB);
 
 
     killJob(job);
     killJob(job);
@@ -4085,17 +4036,18 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   
   
   private static final Counters EMPTY_COUNTERS = new Counters();
   private static final Counters EMPTY_COUNTERS = new Counters();
   public Counters getJobCounters(JobID jobid) throws IOException {
   public Counters getJobCounters(JobID jobid) throws IOException {
+    UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
     synchronized (this) {
     synchronized (this) {
       JobInProgress job = jobs.get(jobid);
       JobInProgress job = jobs.get(jobid);
       if (job != null) {
       if (job != null) {
 
 
         // check the job-access
         // check the job-access
-        job.checkAccess(UserGroupInformation.getCurrentUser(),
-            JobACL.VIEW_JOB);
+        aclsManager.checkAccess(job, callerUGI, null, JobACL.VIEW_JOB);
 
 
         return isJobInited(job) ? job.getCounters() : EMPTY_COUNTERS;
         return isJobInited(job) ? job.getCounters() : EMPTY_COUNTERS;
       } 
       } 
     }
     }
+
     return completedJobStatusStore.readCounters(jobid);
     return completedJobStatusStore.readCounters(jobid);
   }
   }
   
   
@@ -4106,7 +4058,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
       // Check authorization
       // Check authorization
-      job.checkAccess(UserGroupInformation.getCurrentUser(),
+      aclsManager.checkAccess(job, UserGroupInformation.getCurrentUser(), null,
           JobACL.VIEW_JOB);
           JobACL.VIEW_JOB);
     }
     }
     if (job == null || !isJobInited(job)) {
     if (job == null || !isJobInited(job)) {
@@ -4134,7 +4086,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
       // Check authorization
       // Check authorization
-      job.checkAccess(UserGroupInformation.getCurrentUser(),
+      aclsManager.checkAccess(job, UserGroupInformation.getCurrentUser(), null,
           JobACL.VIEW_JOB);
           JobACL.VIEW_JOB);
     }
     }
     if (job == null || !isJobInited(job)) {
     if (job == null || !isJobInited(job)) {
@@ -4160,7 +4112,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
       // Check authorization
       // Check authorization
-      job.checkAccess(UserGroupInformation.getCurrentUser(),
+      aclsManager.checkAccess(job, UserGroupInformation.getCurrentUser(), null,
           JobACL.VIEW_JOB);
           JobACL.VIEW_JOB);
     }
     }
     if (job == null || !isJobInited(job)) {
     if (job == null || !isJobInited(job)) {
@@ -4189,7 +4141,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
       // Check authorization
       // Check authorization
-      job.checkAccess(UserGroupInformation.getCurrentUser(),
+      aclsManager.checkAccess(job, UserGroupInformation.getCurrentUser(), null,
           JobACL.VIEW_JOB);
           JobACL.VIEW_JOB);
     }
     }
     if (job == null || !isJobInited(job)) {
     if (job == null || !isJobInited(job)) {
@@ -4256,7 +4208,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     JobInProgress job = jobs.get(jobId);
     JobInProgress job = jobs.get(jobId);
     if (job != null) {
     if (job != null) {
       // Check authorization
       // Check authorization
-      job.checkAccess(UserGroupInformation.getCurrentUser(),
+      aclsManager.checkAccess(job, UserGroupInformation.getCurrentUser(), null,
           JobACL.VIEW_JOB);
           JobACL.VIEW_JOB);
     }
     }
     if (job != null && isJobInited(job)) {
     if (job != null && isJobInited(job)) {
@@ -4316,7 +4268,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     TaskInProgress tip = taskidToTIPMap.get(taskid);
     TaskInProgress tip = taskidToTIPMap.get(taskid);
     if(tip != null) {
     if(tip != null) {
       // check both queue-level and job-level access
       // check both queue-level and job-level access
-      checkAccess(tip.getJob(), UserGroupInformation.getCurrentUser(),
+      aclsManager.checkAccess(tip.getJob(),
+          UserGroupInformation.getCurrentUser(),
           QueueManager.QueueOperation.ADMINISTER_JOBS, JobACL.MODIFY_JOB);
           QueueManager.QueueOperation.ADMINISTER_JOBS, JobACL.MODIFY_JOB);
 
 
       return tip.killTask(taskid, shouldFail);
       return tip.killTask(taskid, shouldFail);
@@ -4388,7 +4341,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     if (job != null) {
     if (job != null) {
 
 
       // check both queue-level and job-level access
       // check both queue-level and job-level access
-      checkAccess(job, UserGroupInformation.getCurrentUser(),
+      aclsManager.checkAccess(job, UserGroupInformation.getCurrentUser(),
           QueueManager.QueueOperation.ADMINISTER_JOBS, JobACL.MODIFY_JOB);
           QueueManager.QueueOperation.ADMINISTER_JOBS, JobACL.MODIFY_JOB);
 
 
       synchronized (taskScheduler) {
       synchronized (taskScheduler) {
@@ -4579,24 +4532,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       removeMarkedTasks(trackerName);
       removeMarkedTasks(trackerName);
     }
     }
   }
   }
-  
-  /**
-   * Is the calling user a super user? Or part of the supergroup?
-   * @return true, if it is a super user
-   */
-  static boolean isSuperUserOrSuperGroup(UserGroupInformation callerUGI,
-      UserGroupInformation superUser, String superGroup) {
-    if (superUser.getShortUserName().equals(callerUGI.getShortUserName())) {
-      return true;
-    }
-    String[] groups = callerUGI.getGroupNames();
-    for(int i=0; i < groups.length; ++i) {
-      if (groups[i].equals(superGroup)) {
-        return true;
-      }
-    }
-    return false;
-  }
 
 
   /**
   /**
    * Rereads the config to get hosts and exclude list file names.
    * Rereads the config to get hosts and exclude list file names.
@@ -4605,10 +4540,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 (!isSuperUserOrSuperGroup(UserGroupInformation.getCurrentUser(), mrOwner,
-                                 supergroup)) {
+    if (!isMRAdmin(UserGroupInformation.getCurrentUser())) {
       AuditLogger.logFailure(user, Constants.REFRESH_NODES, 
       AuditLogger.logFailure(user, Constants.REFRESH_NODES, 
-          mrOwner + " " + supergroup, Constants.JOBTRACKER, 
+          getMROwner() + " " + getSuperGroup(), 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.");
@@ -4618,15 +4552,19 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     // call the actual api
     // call the actual api
     refreshHosts();
     refreshHosts();
   }
   }
-  
+
   UserGroupInformation getMROwner() {
   UserGroupInformation getMROwner() {
-    return mrOwner;
+    return aclsManager.getMROwner();
   }
   }
 
 
   String getSuperGroup() {
   String getSuperGroup() {
-    return supergroup;
+    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
@@ -4992,6 +4930,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   }
   }
 
 
   JobACLsManager getJobACLsManager() {
   JobACLsManager getJobACLsManager() {
-    return jobACLsManager;
+    return aclsManager.getJobACLsManager();
+  }
+
+  ACLsManager getACLsManager() {
+    return aclsManager;
   }
   }
+
 }
 }

+ 0 - 49
src/mapred/org/apache/hadoop/mapred/JobTrackerJobACLsManager.java

@@ -1,49 +0,0 @@
-/**
- * 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.mapred;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/**
- * Manages the job ACLs and the operations on them at JobTracker.
- *
- */
-public class JobTrackerJobACLsManager extends JobACLsManager {
-
-  static final Log LOG = LogFactory.getLog(JobTrackerJobACLsManager.class);
-
-  private JobTracker jobTracker = null;
-
-  public JobTrackerJobACLsManager(JobTracker tracker) {
-    jobTracker = tracker;
-  }
-
-  @Override
-  protected boolean isJobLevelAuthorizationEnabled() {
-    return jobTracker.isJobLevelAuthorizationEnabled();
-  }
-
-  @Override
-  protected boolean isSuperUserOrSuperGroup(UserGroupInformation callerUGI) {
-    return JobTracker.isSuperUserOrSuperGroup(callerUGI,
-        jobTracker.getMROwner(), jobTracker.getSuperGroup());
-  }
-
-}

+ 10 - 60
src/mapred/org/apache/hadoop/mapred/QueueManager.java

@@ -19,14 +19,11 @@
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.io.PrintWriter;
 import java.io.Writer;
 import java.io.Writer;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Set;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.TreeSet;
-import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -75,28 +72,23 @@ class QueueManager {
    * Enum representing an operation that can be performed on a queue.
    * Enum representing an operation that can be performed on a queue.
    */
    */
   static enum QueueOperation {
   static enum QueueOperation {
-    SUBMIT_JOB ("acl-submit-job", false),
-    ADMINISTER_JOBS ("acl-administer-jobs", true);
+    SUBMIT_JOB ("acl-submit-job"),
+    ADMINISTER_JOBS ("acl-administer-jobs");
     // TODO: Add ACL for LIST_JOBS when we have ability to authenticate 
     // TODO: Add ACL for LIST_JOBS when we have ability to authenticate 
     //       users in UI
     //       users in UI
     // TODO: Add ACL for CHANGE_ACL when we have an admin tool for 
     // TODO: Add ACL for CHANGE_ACL when we have an admin tool for 
     //       configuring queues.
     //       configuring queues.
     
     
     private final String aclName;
     private final String aclName;
-    private final boolean jobOwnerAllowed;
     
     
-    QueueOperation(String aclName, boolean jobOwnerAllowed) {
+    QueueOperation(String aclName) {
       this.aclName = aclName;
       this.aclName = aclName;
-      this.jobOwnerAllowed = jobOwnerAllowed;
     }
     }
 
 
     final String getAclName() {
     final String getAclName() {
       return aclName;
       return aclName;
     }
     }
     
     
-    final boolean isJobOwnerAllowed() {
-      return jobOwnerAllowed;
-    }
   }
   }
   
   
   /**
   /**
@@ -126,7 +118,7 @@ class QueueManager {
   }
   }
   
   
   /**
   /**
-   * Return true if the given {@link QueueManager.QueueOperation} can be 
+   * Return true if the given {@link QueueOperation} can be 
    * performed by the specified user on the given queue.
    * performed by the specified user on the given queue.
    * 
    * 
    * An operation is allowed if all users are provided access for this
    * An operation is allowed if all users are provided access for this
@@ -139,37 +131,9 @@ class QueueManager {
    * 
    * 
    * @return true if the operation is allowed, false otherwise.
    * @return true if the operation is allowed, false otherwise.
    */
    */
-  public synchronized boolean hasAccess(String queueName, QueueOperation oper,
-                                UserGroupInformation ugi) {
-    return hasAccess(queueName, null, oper, ugi);
-  }
-  
-  /**
-   * Return true if the given {@link QueueManager.QueueOperation} can be 
-   * performed by the specified user on the specified job in the given queue.
-   * 
-   * An operation is allowed either if the owner of the job is the user 
-   * performing the task, all users are provided access for this
-   * operation, or if either the user or any of the groups specified is
-   * provided access.
-   * 
-   * If the {@link QueueManager.QueueOperation} is not job specific then the 
-   * job parameter is ignored.
-   * 
-   * @param queueName Queue on which the operation needs to be performed.
-   * @param job The {@link JobInProgress} on which the operation is being
-   *            performed. 
-   * @param oper The operation to perform
-   * @param ugi The user and groups who wish to perform the operation.
-   * 
-   * @return true if the operation is allowed, false otherwise.
-   */
-  public synchronized boolean hasAccess(String queueName, JobInProgress job, 
+  public synchronized boolean hasAccess(String queueName,
                                 QueueOperation oper, 
                                 QueueOperation oper, 
                                 UserGroupInformation ugi) {
                                 UserGroupInformation ugi) {
-    String user = ugi.getShortUserName();
-    String jobId = job == null ? "-" : job.getJobID().toString();
-    
     if (!aclsEnabled) {
     if (!aclsEnabled) {
       return true;
       return true;
     }
     }
@@ -179,17 +143,9 @@ class QueueManager {
                                             oper.getAclName()));      
                                             oper.getAclName()));      
     }
     }
     
     
-    if (oper.isJobOwnerAllowed()) {
-      if (job != null && job.getJobConf().getUser().equals(ugi.getShortUserName())) {
-        AuditLogger.logSuccess(user, oper.name(), queueName);
-        return true;
-      }
-    }
-    
-    AccessControlList acl = aclsMap.get(toFullPropertyName(queueName, oper.getAclName()));
+    AccessControlList acl = aclsMap.get(toFullPropertyName(
+        queueName, oper.getAclName()));
     if (acl == null) {
     if (acl == null) {
-      AuditLogger.logFailure(user, oper.name(), null, queueName, 
-                             "Disabled queue ACLs, job : " + jobId);
       return false;
       return false;
     }
     }
     
     
@@ -201,12 +157,6 @@ class QueueManager {
         allowed = true;
         allowed = true;
       }
       }
     }
     }
-    if (allowed) {
-      AuditLogger.logSuccess(user, oper.name(), queueName);
-    } else {
-      AuditLogger.logFailure(user, oper.name(), null, queueName,
-                             Constants.UNAUTHORIZED_USER + ", job : " + jobId);
-    }
     
     
     return allowed;    
     return allowed;    
   }
   }
@@ -286,7 +236,7 @@ class QueueManager {
     for (String queue : queueNames) {
     for (String queue : queueNames) {
       for (QueueOperation oper : QueueOperation.values()) {
       for (QueueOperation oper : QueueOperation.values()) {
         String key = toFullPropertyName(queue, oper.getAclName());
         String key = toFullPropertyName(queue, oper.getAclName());
-        String aclString = conf.get(key, "*");
+        String aclString = conf.get(key, " ");// default is empty list of users
         aclsMap.put(key, new AccessControlList(aclString));
         aclsMap.put(key, new AccessControlList(aclString));
       }
       }
     } 
     } 
@@ -294,14 +244,14 @@ class QueueManager {
   }
   }
   
   
   private void initialize(Configuration conf) {
   private void initialize(Configuration conf) {
-    aclsEnabled = conf.getBoolean("mapred.acls.enabled", false);
+    aclsEnabled = conf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
     String[] queues = conf.getStrings("mapred.queue.names", 
     String[] queues = conf.getStrings("mapred.queue.names", 
         new String[] {JobConf.DEFAULT_QUEUE_NAME});
         new String[] {JobConf.DEFAULT_QUEUE_NAME});
     addToSet(queueNames, queues);
     addToSet(queueNames, queues);
     aclsMap = getQueueAcls(conf);
     aclsMap = getQueueAcls(conf);
   }
   }
   
   
-  private static final String toFullPropertyName(String queue, 
+  static final String toFullPropertyName(String queue, 
       String property) {
       String property) {
     return QUEUE_CONF_PROPERTY_NAME_PREFIX + queue + "." + property;
     return QUEUE_CONF_PROPERTY_NAME_PREFIX + queue + "." + property;
   }
   }

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

@@ -117,10 +117,10 @@ public class TaskLogServlet extends HttpServlet {
    * 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.
    */
    */
-  private void checkAccessForTaskLogs(JobConf conf, String user, JobID jobId,
+  private void checkAccessForTaskLogs(JobConf conf, String user, String jobId,
       TaskTracker tracker) throws AccessControlException {
       TaskTracker tracker) throws AccessControlException {
 
 
-    if (!tracker.isJobLevelAuthorizationEnabled()) {
+    if (!tracker.areACLsEnabled()) {
       return;
       return;
     }
     }
 
 
@@ -132,7 +132,7 @@ public class TaskLogServlet extends HttpServlet {
     UserGroupInformation callerUGI =
     UserGroupInformation callerUGI =
         UserGroupInformation.createRemoteUser(user);
         UserGroupInformation.createRemoteUser(user);
 
 
-    tracker.getJobACLsManager().checkAccess(jobId, callerUGI, JobACL.VIEW_JOB,
+    tracker.getACLsManager().checkAccess(jobId, callerUGI, JobACL.VIEW_JOB,
         jobOwner, jobViewACL);
         jobOwner, jobViewACL);
   }
   }
 
 
@@ -228,7 +228,7 @@ public class TaskLogServlet extends HttpServlet {
       Configuration jobACLConf = getConfFromJobACLsFile(attemptId, isCleanup);
       Configuration jobACLConf = getConfFromJobACLsFile(attemptId, isCleanup);
       // Ignore authorization if job-acls.xml is not found
       // Ignore authorization if job-acls.xml is not found
       if (jobACLConf != null) {
       if (jobACLConf != null) {
-        JobID jobId = attemptId.getJobID();
+        String jobId = attemptId.getJobID().toString();
 
 
         try {
         try {
           checkAccessForTaskLogs(new JobConf(jobACLConf), user, jobId,
           checkAccessForTaskLogs(new JobConf(jobACLConf), user, jobId,

+ 11 - 8
src/mapred/org/apache/hadoop/mapred/TaskRunner.java

@@ -291,8 +291,11 @@ abstract class TaskRunner extends Thread {
       Localizer.PermissionsHandler.setPermissions(logDir,
       Localizer.PermissionsHandler.setPermissions(logDir,
           Localizer.PermissionsHandler.sevenZeroZero);
           Localizer.PermissionsHandler.sevenZeroZero);
     }
     }
-    // write job acls into a file to know the access for task logs
-    writeJobACLs(logDir);
+
+    if (tracker.areACLsEnabled()) {
+      // write job acls into a file to know the access for task logs
+      writeJobACLs(logDir);
+    }
     return logFiles;
     return logFiles;
   }
   }
 
 
@@ -301,12 +304,12 @@ abstract class TaskRunner extends Thread {
     File aclFile = new File(logDir, TaskRunner.jobACLsFile);
     File aclFile = new File(logDir, TaskRunner.jobACLsFile);
     Configuration aclConf = new Configuration(false);
     Configuration aclConf = new Configuration(false);
 
 
-    // set the job view acls in aclConf
-    String jobViewACLs = conf.get(JobContext.JOB_ACL_VIEW_JOB);
-    if (jobViewACLs != null) {
-      aclConf.set(JobContext.JOB_ACL_VIEW_JOB, jobViewACLs);
-    }
-    // set jobOwner as mapreduce.job.user.name in aclConf
+    // set the job view acl in aclConf
+    String jobViewACL = conf.get(JobContext.JOB_ACL_VIEW_JOB, " ");
+
+    aclConf.set(JobContext.JOB_ACL_VIEW_JOB, jobViewACL);
+
+    // set jobOwner as user.name in aclConf
     String jobOwner = conf.getUser();
     String jobOwner = conf.getUser();
     aclConf.set("user.name", jobOwner);
     aclConf.set("user.name", jobOwner);
     FileOutputStream out = new FileOutputStream(aclFile);
     FileOutputStream out = new FileOutputStream(aclFile);

+ 23 - 27
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -247,9 +247,7 @@ public class TaskTracker
   private int maxReduceSlots;
   private int maxReduceSlots;
   private int failures;
   private int failures;
 
 
-  // MROwner's ugi
-  private UserGroupInformation mrOwner;
-  private String supergroup;
+  private ACLsManager aclsManager;
   
   
   // Performance-related config knob to send an out-of-band heartbeat
   // Performance-related config knob to send an out-of-band heartbeat
   // on task completion
   // on task completion
@@ -278,9 +276,6 @@ public class TaskTracker
   static final String MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY =
   static final String MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY =
       "mapred.tasktracker.memory_calculator_plugin";
       "mapred.tasktracker.memory_calculator_plugin";
 
 
-  // Manages job acls of jobs in TaskTracker
-  private TaskTrackerJobACLsManager jobACLsManager;
-
   /**
   /**
    * the minimum interval between jobtracker polls
    * the minimum interval between jobtracker polls
    */
    */
@@ -585,16 +580,11 @@ public class TaskTracker
     this.fConf = new JobConf(originalConf);
     this.fConf = new JobConf(originalConf);
     UserGroupInformation.setConfiguration(fConf);
     UserGroupInformation.setConfiguration(fConf);
     SecurityUtil.login(fConf, TT_KEYTAB_FILE, TT_USER_NAME);
     SecurityUtil.login(fConf, TT_KEYTAB_FILE, TT_USER_NAME);
-    if (UserGroupInformation.isLoginKeytabBased()) {
-      mrOwner = UserGroupInformation.getLoginUser();
-    } else {
-      mrOwner = UserGroupInformation.getCurrentUser();
-    }
 
 
-    supergroup = fConf.get(JobConf.MR_SUPERGROUP,
-                           "supergroup");
-    LOG.info("Starting tasktracker with owner as " + mrOwner.getShortUserName()
-             + " and supergroup as " + supergroup);
+    aclsManager = new ACLsManager(fConf, new JobACLsManager(fConf), null);
+    LOG.info("Starting tasktracker with owner as " +
+        getMROwner().getShortUserName() + " and supergroup as " +
+        getSuperGroup());
 
 
     localFs = FileSystem.getLocal(fConf);
     localFs = FileSystem.getLocal(fConf);
     if (fConf.get("slave.host.name") != null) {
     if (fConf.get("slave.host.name") != null) {
@@ -691,7 +681,7 @@ public class TaskTracker
         this.fConf, taskController);
         this.fConf, taskController);
 
 
     this.jobClient = (InterTrackerProtocol) 
     this.jobClient = (InterTrackerProtocol) 
-    mrOwner.doAs(new PrivilegedExceptionAction<Object>() {
+    getMROwner().doAs(new PrivilegedExceptionAction<Object>() {
       public Object run() throws IOException {
       public Object run() throws IOException {
         return RPC.waitForProxy(InterTrackerProtocol.class,
         return RPC.waitForProxy(InterTrackerProtocol.class,
             InterTrackerProtocol.versionID,
             InterTrackerProtocol.versionID,
@@ -732,19 +722,22 @@ public class TaskTracker
   }
   }
 
 
   UserGroupInformation getMROwner() {
   UserGroupInformation getMROwner() {
-    return mrOwner;
+    return aclsManager.getMROwner();
   }
   }
 
 
   String getSuperGroup() {
   String getSuperGroup() {
-    return supergroup;
+    return aclsManager.getSuperGroup();
   }
   }
-  
+
+  boolean isMRAdmin(UserGroupInformation ugi) {
+    return aclsManager.isMRAdmin(ugi);
+  }
+
   /**
   /**
-   * Is job level authorization enabled on the TT ?
+   * Are ACLs for authorization checks enabled on the TT ?
    */
    */
-  boolean isJobLevelAuthorizationEnabled() {
-    return fConf.getBoolean(
-        JobConf.JOB_LEVEL_AUTHORIZATION_ENABLING_FLAG, false);
+  boolean areACLsEnabled() {
+    return fConf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
   }
   }
 
 
   public static Class<? extends TaskTrackerInstrumentation> getInstrumentationClass(
   public static Class<? extends TaskTrackerInstrumentation> getInstrumentationClass(
@@ -1257,8 +1250,7 @@ public class TaskTracker
     checkJettyPort(httpPort);
     checkJettyPort(httpPort);
     // create user log manager
     // create user log manager
     setUserLogManager(new UserLogManager(conf));
     setUserLogManager(new UserLogManager(conf));
-    // Initialize the jobACLSManager
-    jobACLsManager = new TaskTrackerJobACLsManager(this);
+
     initialize();
     initialize();
   }
   }
 
 
@@ -3823,7 +3815,11 @@ public class TaskTracker
       return localJobTokenFileStr;
       return localJobTokenFileStr;
     }
     }
 
 
-    TaskTrackerJobACLsManager getJobACLsManager() {
-      return jobACLsManager;
+    JobACLsManager getJobACLsManager() {
+      return aclsManager.getJobACLsManager();
+    }
+    
+    ACLsManager getACLsManager() {
+      return aclsManager;
     }
     }
 }
 }

+ 0 - 48
src/mapred/org/apache/hadoop/mapred/TaskTrackerJobACLsManager.java

@@ -1,48 +0,0 @@
-/**
- * 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.mapred;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/**
- * Manages the job ACLs and the operations on them at TaskTracker.
- *
- */
-public class TaskTrackerJobACLsManager extends JobACLsManager {
-
-  static final Log LOG = LogFactory.getLog(TaskTrackerJobACLsManager.class);
-
-  private TaskTracker taskTracker = null;
-
-  public TaskTrackerJobACLsManager(TaskTracker tracker) {
-    taskTracker = tracker;
-  }
-
-  @Override
-  protected boolean isJobLevelAuthorizationEnabled() {
-    return taskTracker.isJobLevelAuthorizationEnabled();
-  }
-
-  @Override
-  protected boolean isSuperUserOrSuperGroup(UserGroupInformation callerUGI) {
-    return JobTracker.isSuperUserOrSuperGroup(callerUGI,
-        taskTracker.getMROwner(), taskTracker.getSuperGroup());
-  }
-}

+ 6 - 2
src/test/org/apache/hadoop/mapred/TestJobACLs.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.mapred.JobPriority;
 import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapred.JobTracker;
 import org.apache.hadoop.mapred.JobTracker;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.QueueManager.QueueOperation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -70,8 +71,11 @@ public class TestJobACLs {
     UserGroupInformation MR_UGI = UserGroupInformation.getLoginUser();
     UserGroupInformation MR_UGI = UserGroupInformation.getLoginUser();
     JobConf conf = new JobConf();
     JobConf conf = new JobConf();
 
 
-    // Enable job-level authorization
-    conf.setBoolean(JobConf.JOB_LEVEL_AUTHORIZATION_ENABLING_FLAG, true);
+    // Enable queue and job level authorization
+    conf.setBoolean(JobConf.MR_ACLS_ENABLED, true);
+    // no queue admins for default queue
+    conf.set(QueueManager.toFullPropertyName(
+        "default", QueueOperation.ADMINISTER_JOBS.getAclName()), " ");
 
 
     // Enable CompletedJobStore
     // Enable CompletedJobStore
     FileSystem fs = FileSystem.getLocal(conf);
     FileSystem fs = FileSystem.getLocal(conf);

+ 6 - 2
src/test/org/apache/hadoop/mapred/TestJobHistory.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapred.JobHistory.*;
 import org.apache.hadoop.mapred.JobHistory.*;
+import org.apache.hadoop.mapred.QueueManager.QueueOperation;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -801,7 +802,7 @@ public class TestJobHistory extends TestCase {
     validateTaskAttemptLevelKeyValues(mr, job, jobInfo);
     validateTaskAttemptLevelKeyValues(mr, job, jobInfo);
 
 
     // Also JobACLs should be correct
     // Also JobACLs should be correct
-    if (mr.getJobTrackerRunner().getJobTracker().isJobLevelAuthorizationEnabled()) {
+    if (mr.getJobTrackerRunner().getJobTracker().areACLsEnabled()) {
       assertEquals(conf.get(JobACL.VIEW_JOB.getAclName()),
       assertEquals(conf.get(JobACL.VIEW_JOB.getAclName()),
           jobInfo.getJobACLs().get(JobACL.VIEW_JOB).toString());
           jobInfo.getJobACLs().get(JobACL.VIEW_JOB).toString());
       assertEquals(conf.get(JobACL.MODIFY_JOB.getAclName()),
       assertEquals(conf.get(JobACL.MODIFY_JOB.getAclName()),
@@ -911,7 +912,10 @@ public class TestJobHistory extends TestCase {
       conf.set("mapred.job.tracker.history.completed.location", doneFolder);
       conf.set("mapred.job.tracker.history.completed.location", doneFolder);
 
 
       // Enable ACLs so that they are logged to history
       // Enable ACLs so that they are logged to history
-      conf.setBoolean(JobConf.JOB_LEVEL_AUTHORIZATION_ENABLING_FLAG, true);
+      conf.setBoolean(JobConf.MR_ACLS_ENABLED, true);
+      // no queue admins for default queue
+      conf.set(QueueManager.toFullPropertyName(
+          "default", QueueOperation.ADMINISTER_JOBS.getAclName()), " ");
       
       
       mr = new MiniMRCluster(2, "file:///", 3, null, null, conf);
       mr = new MiniMRCluster(2, "file:///", 3, null, null, conf);
 
 

+ 4 - 2
src/test/org/apache/hadoop/mapred/TestJobTrackerRestart.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapred.UtilsForTests;
 import org.apache.hadoop.mapred.UtilsForTests;
+import org.apache.hadoop.mapred.QueueManager.QueueOperation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
@@ -531,10 +532,11 @@ public class TestJobTrackerRestart extends TestCase {
       jtConf.set("mapred.jobtracker.job.history.buffer.size", "1024");
       jtConf.set("mapred.jobtracker.job.history.buffer.size", "1024");
       jtConf.setInt("mapred.tasktracker.reduce.tasks.maximum", 1);
       jtConf.setInt("mapred.tasktracker.reduce.tasks.maximum", 1);
       jtConf.setLong("mapred.tasktracker.expiry.interval", 25 * 1000);
       jtConf.setLong("mapred.tasktracker.expiry.interval", 25 * 1000);
-      jtConf.setBoolean("mapred.acls.enabled", true);
+      jtConf.setBoolean(JobConf.MR_ACLS_ENABLED, true);
       // get the user group info
       // get the user group info
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-      jtConf.set("mapred.queue.default.acl-submit-job", ugi.getUserName());
+      jtConf.set(QueueManager.toFullPropertyName("default",
+          QueueOperation.SUBMIT_JOB.getAclName()), ugi.getUserName());
       
       
       mr = new MiniMRCluster(1, namenode, 1, null, null, jtConf);
       mr = new MiniMRCluster(1, namenode, 1, null, null, jtConf);
       
       

+ 27 - 24
src/test/org/apache/hadoop/mapred/TestQueueAclsForCurrentUser.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.mapred;
 import java.io.IOException;
 import java.io.IOException;
 import javax.security.auth.login.LoginException;
 import javax.security.auth.login.LoginException;
 import junit.framework.TestCase;
 import junit.framework.TestCase;
+
+import org.apache.hadoop.mapred.QueueManager.QueueOperation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
 /**
 /**
@@ -31,23 +33,23 @@ public class TestQueueAclsForCurrentUser extends TestCase {
   private QueueManager queueManager;
   private QueueManager queueManager;
   private JobConf conf = null;
   private JobConf conf = null;
   UserGroupInformation currentUGI = null;
   UserGroupInformation currentUGI = null;
-  String submitAcl = QueueManager.QueueOperation.SUBMIT_JOB.getAclName();
-  String adminAcl  = QueueManager.QueueOperation.ADMINISTER_JOBS.getAclName();
+  String submitAcl = QueueOperation.SUBMIT_JOB.getAclName();
+  String adminAcl  = QueueOperation.ADMINISTER_JOBS.getAclName();
 
 
   private void setupConfForNoAccess() throws IOException,LoginException {
   private void setupConfForNoAccess() throws IOException,LoginException {
     currentUGI = UserGroupInformation.getLoginUser();
     currentUGI = UserGroupInformation.getLoginUser();
     String userName = currentUGI.getUserName();
     String userName = currentUGI.getUserName();
     conf = new JobConf();
     conf = new JobConf();
 
 
-    conf.setBoolean("mapred.acls.enabled",true);
+    conf.setBoolean(JobConf.MR_ACLS_ENABLED,true);
 
 
     conf.set("mapred.queue.names", "qu1,qu2");
     conf.set("mapred.queue.names", "qu1,qu2");
     //Only user u1 has access
     //Only user u1 has access
-    conf.set("mapred.queue.qu1.acl-submit-job", "u1");
-    conf.set("mapred.queue.qu1.acl-administer-jobs", "u1");
+    conf.set(QueueManager.toFullPropertyName("qu1", submitAcl), "u1");
+    conf.set(QueueManager.toFullPropertyName("qu1", adminAcl), "u1");
     //q2 only group g2 has acls for the queues
     //q2 only group g2 has acls for the queues
-    conf.set("mapred.queue.qu2.acl-submit-job", " g2");
-    conf.set("mapred.queue.qu2.acl-administer-jobs", " g2");
+    conf.set(QueueManager.toFullPropertyName("qu2", submitAcl), " g2");
+    conf.set(QueueManager.toFullPropertyName("qu2", adminAcl), " g2");
     queueManager = new QueueManager(conf);
     queueManager = new QueueManager(conf);
 
 
   }
   }
@@ -61,27 +63,27 @@ public class TestQueueAclsForCurrentUser extends TestCase {
     String userName = currentUGI.getUserName();
     String userName = currentUGI.getUserName();
     conf = new JobConf();
     conf = new JobConf();
 
 
-    conf.setBoolean("mapred.acls.enabled", aclSwitch);
+    conf.setBoolean(JobConf.MR_ACLS_ENABLED, aclSwitch);
 
 
     conf.set("mapred.queue.names", "qu1,qu2,qu3,qu4,qu5,qu6,qu7");
     conf.set("mapred.queue.names", "qu1,qu2,qu3,qu4,qu5,qu6,qu7");
     //q1 Has acls for all the users, supports both submit and administer
     //q1 Has acls for all the users, supports both submit and administer
-    conf.set("mapred.queue.qu1.acl-submit-job", "*");
-    conf.set("mapred.queue.qu1-acl-administer-jobs", "*");
+    conf.set(QueueManager.toFullPropertyName("qu1", submitAcl), "*");
+    conf.set(QueueManager.toFullPropertyName("qu1", adminAcl), "*");
     //q2 only u2 has acls for the queues
     //q2 only u2 has acls for the queues
-    conf.set("mapred.queue.qu2.acl-submit-job", "u2");
-    conf.set("mapred.queue.qu2.acl-administer-jobs", "u2");
+    conf.set(QueueManager.toFullPropertyName("qu2", submitAcl), "u2");
+    conf.set(QueueManager.toFullPropertyName("qu2", adminAcl), "u2");
     //q3  Only u2 has submit operation access rest all have administer access
     //q3  Only u2 has submit operation access rest all have administer access
-    conf.set("mapred.queue.qu3.acl-submit-job", "u2");
-    conf.set("mapred.queue.qu3.acl-administer-jobs", "*");
+    conf.set(QueueManager.toFullPropertyName("qu3", submitAcl), "u2");
+    conf.set(QueueManager.toFullPropertyName("qu3", adminAcl), "*");
     //q4 Only u2 has administer access , anyone can do submit
     //q4 Only u2 has administer access , anyone can do submit
-    conf.set("mapred.queue.qu4.acl-submit-job", "*");
-    conf.set("mapred.queue.qu4.acl-administer-jobs", "u2");
+    conf.set(QueueManager.toFullPropertyName("qu4", submitAcl), "*");
+    conf.set(QueueManager.toFullPropertyName("qu4", adminAcl), "u2");
     //qu6 only current user has submit access
     //qu6 only current user has submit access
-    conf.set("mapred.queue.qu6.acl-submit-job",userName);
-    conf.set("mapred.queue.qu6.acl-administrator-jobs","u2");
+    conf.set(QueueManager.toFullPropertyName("qu6", submitAcl),userName);
+    conf.set(QueueManager.toFullPropertyName("qu6", adminAcl),"u2");
     //qu7 only current user has administrator access
     //qu7 only current user has administrator access
-    conf.set("mapred.queue.qu7.acl-submit-job","u2");
-    conf.set("mapred.queue.qu7.acl-administrator-jobs",userName);
+    conf.set(QueueManager.toFullPropertyName("qu7", submitAcl),"u2");
+    conf.set(QueueManager.toFullPropertyName("qu7", adminAcl),userName);
     //qu8 only current group has access
     //qu8 only current group has access
     StringBuilder groupNames = new StringBuilder("");
     StringBuilder groupNames = new StringBuilder("");
     String[] ugiGroupNames = currentUGI.getGroupNames();
     String[] ugiGroupNames = currentUGI.getGroupNames();
@@ -92,9 +94,10 @@ public class TestQueueAclsForCurrentUser extends TestCase {
         groupNames.append(",");
         groupNames.append(",");
       }
       }
     }
     }
-    conf.set("mapred.queue.qu5.acl-submit-job"," "+groupNames.toString());
-    conf.set("mapred.queue.qu5.acl-administrator-jobs"," "
-            +groupNames.toString());
+    conf.set(QueueManager.toFullPropertyName("qu5", submitAcl),
+        " " + groupNames.toString());
+    conf.set(QueueManager.toFullPropertyName("qu5", adminAcl),
+        " " + groupNames.toString());
 
 
     queueManager = new QueueManager(conf);
     queueManager = new QueueManager(conf);
   }
   }
@@ -124,7 +127,7 @@ public class TestQueueAclsForCurrentUser extends TestCase {
 
 
   private void checkQueueAclsInfo(QueueAclsInfo[] queueAclsInfoList)
   private void checkQueueAclsInfo(QueueAclsInfo[] queueAclsInfoList)
           throws IOException {
           throws IOException {
-    if (conf.get("mapred.acls.enabled").equalsIgnoreCase("true")) {
+    if (conf.get(JobConf.MR_ACLS_ENABLED).equalsIgnoreCase("true")) {
       for (int i = 0; i < queueAclsInfoList.length; i++) {
       for (int i = 0; i < queueAclsInfoList.length; i++) {
         QueueAclsInfo acls = queueAclsInfoList[i];
         QueueAclsInfo acls = queueAclsInfoList[i];
         String queueName = acls.getQueueName();
         String queueName = acls.getQueueName();

+ 244 - 129
src/test/org/apache/hadoop/mapred/TestQueueManager.java

@@ -38,12 +38,16 @@ import org.apache.hadoop.examples.SleepJob;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.QueueManager.QueueOperation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
 public class TestQueueManager extends TestCase {
 public class TestQueueManager extends TestCase {
 
 
   private static final Log LOG = LogFactory.getLog(TestQueueManager.class);
   private static final Log LOG = LogFactory.getLog(TestQueueManager.class);
-  
+
+  String submitAcl = QueueOperation.SUBMIT_JOB.getAclName();
+  String adminAcl  = QueueOperation.ADMINISTER_JOBS.getAclName();
+
   private MiniDFSCluster miniDFSCluster;
   private MiniDFSCluster miniDFSCluster;
   private MiniMRCluster miniMRCluster;
   private MiniMRCluster miniMRCluster;
   
   
@@ -56,13 +60,12 @@ public class TestQueueManager extends TestCase {
   private UserGroupInformation createNecessaryUsers() throws IOException {
   private UserGroupInformation createNecessaryUsers() throws IOException {
     // Add real user to fake groups mapping so that child processes (tasks)
     // Add real user to fake groups mapping so that child processes (tasks)
     // will have permissions on the dfs
     // will have permissions on the dfs
-    String j = UserGroupInformation.getCurrentUser().getUserName();
-    UserGroupInformation.createUserForTesting(j, new String [] { "supergroup"});
+    String j = UserGroupInformation.getCurrentUser().getShortUserName();
+    UserGroupInformation.createUserForTesting(j, new String [] { "myGroup"});
     
     
-    
-    // Create a fake superuser for all processes to execute within
+    // Create a fake user for all processes to execute within
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting("Zork",
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting("Zork",
-                                                 new String [] {"Zork"});
+                                                 new String [] {"ZorkGroup"});
     return ugi;
     return ugi;
   }
   }
   
   
@@ -73,7 +76,7 @@ public class TestQueueManager extends TestCase {
     expQueues.add("default");
     expQueues.add("default");
     verifyQueues(expQueues, qMgr.getQueues());
     verifyQueues(expQueues, qMgr.getQueues());
     // pass true so it will fail if the key is not found.
     // pass true so it will fail if the key is not found.
-    assertFalse(conf.getBoolean("mapred.acls.enabled", true));
+    assertFalse(conf.getBoolean(JobConf.MR_ACLS_ENABLED, true));
   }
   }
   
   
   public void testMultipleQueues() {
   public void testMultipleQueues() {
@@ -86,7 +89,7 @@ public class TestQueueManager extends TestCase {
     expQueues.add("Q3");
     expQueues.add("Q3");
     verifyQueues(expQueues, qMgr.getQueues());
     verifyQueues(expQueues, qMgr.getQueues());
   }
   }
-  
+
   public void testSchedulerInfo() {
   public void testSchedulerInfo() {
     JobConf conf = new JobConf();
     JobConf conf = new JobConf();
     conf.set("mapred.queue.names", "qq1,qq2");
     conf.set("mapred.queue.names", "qq1,qq2");
@@ -99,35 +102,56 @@ public class TestQueueManager extends TestCase {
   
   
   public void testAllEnabledACLForJobSubmission() 
   public void testAllEnabledACLForJobSubmission() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
-    JobConf conf = setupConf("mapred.queue.default.acl-submit-job", "*");
-    verifyJobSubmission(conf, true);
+    JobConf conf = setupConf(QueueManager.toFullPropertyName(
+        "default", submitAcl), "*");
+    UserGroupInformation ugi = createNecessaryUsers();
+    String[] groups = ugi.getGroupNames();
+    verifyJobSubmissionToDefaultQueue(conf, true,
+        ugi.getShortUserName() + "," + groups[groups.length-1]);
   }
   }
   
   
   public void testAllDisabledACLForJobSubmission() 
   public void testAllDisabledACLForJobSubmission() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
-    JobConf conf = setupConf("mapred.queue.default.acl-submit-job", "");
-    verifyJobSubmission(conf, false);
+    createNecessaryUsers();
+    JobConf conf = setupConf(QueueManager.toFullPropertyName(
+        "default", submitAcl), " ");
+    String userName = "user1";
+    String groupName = "group1";
+    verifyJobSubmissionToDefaultQueue(conf, false, userName + "," + groupName);
+    
+    // Check if member of supergroup can submit job
+    conf.set(JobConf.MR_SUPERGROUP, groupName);
+    verifyJobSubmissionToDefaultQueue(conf, true, userName + "," + groupName);
+    
+    // Check if MROwner(user who started the mapreduce cluster) can submit job
+    UserGroupInformation mrOwner = UserGroupInformation.getCurrentUser();
+    userName = mrOwner.getShortUserName();
+    String[] groups = mrOwner.getGroupNames();
+    groupName = groups[groups.length - 1];
+    verifyJobSubmissionToDefaultQueue(conf, true, userName + "," + groupName);
   }
   }
   
   
   public void testUserDisabledACLForJobSubmission() 
   public void testUserDisabledACLForJobSubmission() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
-    JobConf conf = setupConf("mapred.queue.default.acl-submit-job", 
-                                "3698-non-existent-user");
-    verifyJobSubmission(conf, false);
+    JobConf conf = setupConf(QueueManager.toFullPropertyName(
+        "default", submitAcl), "3698-non-existent-user");
+    verifyJobSubmissionToDefaultQueue(conf, false, "user1,group1");
   }
   }
   
   
   public void testDisabledACLForNonDefaultQueue() 
   public void testDisabledACLForNonDefaultQueue() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
     // allow everyone in default queue
     // allow everyone in default queue
-    JobConf conf = setupConf("mapred.queue.default.acl-submit-job", "*");
+    JobConf conf = setupConf(QueueManager.toFullPropertyName(
+        "default", submitAcl), "*");
     // setup a different queue
     // setup a different queue
     conf.set("mapred.queue.names", "default,q1");
     conf.set("mapred.queue.names", "default,q1");
     // setup a different acl for this queue.
     // setup a different acl for this queue.
-    conf.set("mapred.queue.q1.acl-submit-job", "dummy-user");
+    conf.set(QueueManager.toFullPropertyName(
+        "q1", submitAcl), "dummy-user");
     // verify job submission to other queue fails.
     // verify job submission to other queue fails.
-    verifyJobSubmission(conf, false, "q1");
+    verifyJobSubmission(conf, false, "user1,group1", "q1");
   }
   }
-  
+
   public void testSubmissionToInvalidQueue() 
   public void testSubmissionToInvalidQueue() 
   throws IOException, InterruptedException{
   throws IOException, InterruptedException{
     JobConf conf = new JobConf();
     JobConf conf = new JobConf();
@@ -144,65 +168,62 @@ public class TestQueueManager extends TestCase {
     }
     }
     fail("Job submission to invalid queue job shouldnot complete , it should fail with proper exception ");   
     fail("Job submission to invalid queue job shouldnot complete , it should fail with proper exception ");   
   }
   }
-  
+
   public void testEnabledACLForNonDefaultQueue() 
   public void testEnabledACLForNonDefaultQueue() 
   throws IOException, LoginException, InterruptedException {
   throws IOException, LoginException, InterruptedException {
-    // login as self...
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    String userName = ugi.getUserName();
+    UserGroupInformation ugi = createNecessaryUsers();
+    String[] groups = ugi.getGroupNames();
+    String userName = ugi.getShortUserName();
     // allow everyone in default queue
     // allow everyone in default queue
-    JobConf conf = setupConf("mapred.queue.default.acl-submit-job", "*");
+    JobConf conf = setupConf(QueueManager.toFullPropertyName(
+        "default", submitAcl), "*");
     // setup a different queue
     // setup a different queue
     conf.set("mapred.queue.names", "default,q2");
     conf.set("mapred.queue.names", "default,q2");
     // setup a different acl for this queue.
     // setup a different acl for this queue.
-    conf.set("mapred.queue.q2.acl-submit-job", userName);
+    conf.set(QueueManager.toFullPropertyName(
+        "q2", submitAcl), userName);
     // verify job submission to other queue fails.
     // verify job submission to other queue fails.
-    verifyJobSubmission(conf, true, "q2");
+    verifyJobSubmission(conf, true,
+        userName + "," + groups[groups.length-1], "q2");
   }
   }
-  
+
   public void testUserEnabledACLForJobSubmission() 
   public void testUserEnabledACLForJobSubmission() 
   throws IOException, LoginException, InterruptedException {
   throws IOException, LoginException, InterruptedException {
-    // login as self...
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    String userName = ugi.getUserName();
-    JobConf conf = setupConf("mapred.queue.default.acl-submit-job",
-                                  "3698-junk-user," + userName 
+    String userName = "user1";
+    JobConf conf = setupConf(QueueManager.toFullPropertyName(
+        "default", submitAcl), "3698-junk-user," + userName 
                                     + " 3698-junk-group1,3698-junk-group2");
                                     + " 3698-junk-group1,3698-junk-group2");
-    verifyJobSubmission(conf, true);
+    verifyJobSubmissionToDefaultQueue(conf, true, userName+",group1");
   }
   }
-  
+
   public void testGroupsEnabledACLForJobSubmission() 
   public void testGroupsEnabledACLForJobSubmission() 
   throws IOException, LoginException, InterruptedException {
   throws IOException, LoginException, InterruptedException {
     // login as self, get one group, and add in allowed list.
     // login as self, get one group, and add in allowed list.
     UserGroupInformation ugi = createNecessaryUsers();
     UserGroupInformation ugi = createNecessaryUsers();
-    
-    ugi.doAs(new PrivilegedExceptionAction<Object>() {
-
-      @Override
-      public Object run() throws Exception {
-        String[] groups = UserGroupInformation.getCurrentUser().getGroupNames();
-        JobConf conf = setupConf("mapred.queue.default.acl-submit-job",
-                                    "3698-junk-user1,3698-junk-user2 " 
-                                      + groups[groups.length-1] 
-                                               + ",3698-junk-group");
-        verifyJobSubmission(conf, true);
-        
-        return null;
-      }
-    });
 
 
+    String[] groups = ugi.getGroupNames();
+    JobConf conf = setupConf(QueueManager.toFullPropertyName(
+        "default", submitAcl), "3698-junk-user1,3698-junk-user2 " 
+                               + groups[groups.length-1] 
+                               + ",3698-junk-group");
+    verifyJobSubmissionToDefaultQueue(conf, true,
+        		ugi.getShortUserName()+","+groups[groups.length-1]);
   }
   }
-  
+
   public void testAllEnabledACLForJobKill() 
   public void testAllEnabledACLForJobKill() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
     UserGroupInformation ugi = createNecessaryUsers();
     UserGroupInformation ugi = createNecessaryUsers();
-    
+    // create other user who will try to kill the job of ugi.
+    final UserGroupInformation otherUGI = UserGroupInformation.
+        createUserForTesting("user1", new String [] {"group1"});
+
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
 
 
       @Override
       @Override
       public Object run() throws Exception {
       public Object run() throws Exception {
-        JobConf conf = setupConf("mapred.queue.default.acl-administer-jobs", "*");
-        verifyJobKill(conf, true);
+        JobConf conf = setupConf(QueueManager.toFullPropertyName(
+            "default", adminAcl), "*");
+        verifyJobKill(otherUGI, conf, true);
         return null;
         return null;
       }
       }
     });
     });
@@ -211,16 +232,30 @@ public class TestQueueManager extends TestCase {
   public void testAllDisabledACLForJobKill() 
   public void testAllDisabledACLForJobKill() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
     // Create a fake superuser for all processes to execute within
     // Create a fake superuser for all processes to execute within
-    UserGroupInformation ugi = createNecessaryUsers();
+    final UserGroupInformation ugi = createNecessaryUsers();
+
+    // create other user who will try to kill the job of ugi.
+    final UserGroupInformation otherUGI = UserGroupInformation.
+        createUserForTesting("user1", new String [] {"group1"});
+
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
 
 
       @Override
       @Override
       public Object run() throws Exception {
       public Object run() throws Exception {
-        // No one should be able to kill jobs
-        JobConf conf = setupConf("mapred.queue.default.acl-administer-jobs", "");
-        // Run as dummy-user, who (obviously) is not able to kill the job,
-        // and expect him to fail
-        verifyJobKillAsOtherUser(conf, false, "dummy-user,dummy-group");
+        // No queue admins
+        JobConf conf = setupConf(QueueManager.toFullPropertyName(
+            "default", adminAcl), " ");
+        // Run job as ugi and try to kill job as user1, who (obviously)
+        // should not be able to kill the job.
+        verifyJobKill(otherUGI, conf, false);
+
+        // Check if member of supergroup can kill job
+        conf.set(JobConf.MR_SUPERGROUP, "group1");
+        verifyJobKill(otherUGI, conf, true);
+        
+        // Check if MROwner(user who started the mapreduce cluster) can kill job
+        verifyJobKill(ugi, conf, true);
+
         return null;
         return null;
       }
       }
     });
     });
@@ -228,16 +263,16 @@ public class TestQueueManager extends TestCase {
   
   
   public void testOwnerAllowedForJobKill() 
   public void testOwnerAllowedForJobKill() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
-    UserGroupInformation ugi = createNecessaryUsers();
+    final UserGroupInformation ugi = createNecessaryUsers();
     
     
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
 
 
       @Override
       @Override
       public Object run() throws Exception {
       public Object run() throws Exception {
 
 
-        JobConf conf = setupConf("mapred.queue.default.acl-administer-jobs", 
-                                              "junk-user");
-        verifyJobKill(conf, true);
+        JobConf conf = setupConf(QueueManager.toFullPropertyName(
+            "default", adminAcl), "junk-user");
+        verifyJobKill(ugi, conf, true);
         return null;
         return null;
       }
       }
     });
     });
@@ -246,32 +281,41 @@ public class TestQueueManager extends TestCase {
   public void testUserDisabledACLForJobKill() 
   public void testUserDisabledACLForJobKill() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
     UserGroupInformation ugi = createNecessaryUsers();
     UserGroupInformation ugi = createNecessaryUsers();
-    
+    // create other user who will try to kill the job of ugi.
+    final UserGroupInformation otherUGI = UserGroupInformation.
+        createUserForTesting("user1", new String [] {"group1"});
+
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
+
       @Override
       @Override
       public Object run() throws Exception {
       public Object run() throws Exception {
-      //setup a cluster allowing a user to submit
-        JobConf conf = setupConf("mapred.queue.default.acl-administer-jobs", 
-                                                "dummy-user");
-        verifyJobKillAsOtherUser(conf, false, "dummy-user,dummy-group");
+        //setup a cluster allowing a user to submit
+        JobConf conf = setupConf(QueueManager.toFullPropertyName(
+            "default", adminAcl), "dummy-user");
+        // Run job as ugi and try to kill job as user1, who (obviously)
+        // should not able to kill the job.
+        verifyJobKill(otherUGI, conf, false);
         return null;
         return null;
       }
       }
     });
     });
-   }
+  }
   
   
   public void testUserEnabledACLForJobKill() 
   public void testUserEnabledACLForJobKill() 
   throws IOException, LoginException, InterruptedException {
   throws IOException, LoginException, InterruptedException {
   UserGroupInformation ugi = createNecessaryUsers();
   UserGroupInformation ugi = createNecessaryUsers();
-  
+  // create other user who will try to kill the job of ugi.
+  final UserGroupInformation otherUGI = UserGroupInformation.
+      createUserForTesting("user1", new String [] {"group1"});
+
   ugi.doAs(new PrivilegedExceptionAction<Object>() {
   ugi.doAs(new PrivilegedExceptionAction<Object>() {
     @Override
     @Override
     public Object run() throws Exception {
     public Object run() throws Exception {
-      // login as self...
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-      String userName = ugi.getUserName();
-      JobConf conf = setupConf("mapred.queue.default.acl-administer-jobs",
-                                                "dummy-user,"+userName);
-      verifyJobKillAsOtherUser(conf, true, "dummy-user,dummy-group");
+      String userName = ugi.getShortUserName();
+      JobConf conf = setupConf(QueueManager.toFullPropertyName(
+          "default", adminAcl), "user1");
+      // user1 should be able to kill the job
+      verifyJobKill(otherUGI, conf, true);
       return null;
       return null;
       }
       }
     });
     });
@@ -280,15 +324,18 @@ public class TestQueueManager extends TestCase {
   public void testUserDisabledForJobPriorityChange() 
   public void testUserDisabledForJobPriorityChange() 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
     UserGroupInformation ugi = createNecessaryUsers();
     UserGroupInformation ugi = createNecessaryUsers();
+    // create other user who will try to change priority of the job of ugi.
+    final UserGroupInformation otherUGI = UserGroupInformation.
+        createUserForTesting("user1", new String [] {"group1"});
+
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
 
 
       @Override
       @Override
       public Object run() throws Exception {
       public Object run() throws Exception {
 
 
-        JobConf conf = setupConf("mapred.queue.default.acl-administer-jobs",
-                              "junk-user");
-        verifyJobPriorityChangeAsOtherUser(conf, false, 
-                              "junk-user,dummy-group");
+        JobConf conf = setupConf(QueueManager.toFullPropertyName(
+            "default", adminAcl), "junk-user");
+        verifyJobPriorityChangeAsOtherUser(otherUGI, conf, false);
         return null;
         return null;
       }
       }
     });
     });
@@ -310,23 +357,29 @@ public class TestQueueManager extends TestCase {
       Properties hadoopConfProps = new Properties();
       Properties hadoopConfProps = new Properties();
       //these properties should be retained.
       //these properties should be retained.
       hadoopConfProps.put("mapred.queue.names", "default,q1,q2");
       hadoopConfProps.put("mapred.queue.names", "default,q1,q2");
-      hadoopConfProps.put("mapred.acls.enabled", "true");
+      hadoopConfProps.put(JobConf.MR_ACLS_ENABLED, "true");
       //These property should always be overridden
       //These property should always be overridden
-      hadoopConfProps.put("mapred.queue.default.acl-submit-job", "u1");
-      hadoopConfProps.put("mapred.queue.q1.acl-submit-job", "u2");
-      hadoopConfProps.put("mapred.queue.q2.acl-submit-job", "u1");
+      hadoopConfProps.put(QueueManager.toFullPropertyName(
+          "default", submitAcl), "u1");
+      hadoopConfProps.put(QueueManager.toFullPropertyName(
+          "q1", submitAcl), "u2");
+      hadoopConfProps.put(QueueManager.toFullPropertyName(
+          "q2", submitAcl), "u1");
       UtilsForTests.setUpConfigFile(hadoopConfProps, hadoopConfigFile);
       UtilsForTests.setUpConfigFile(hadoopConfProps, hadoopConfigFile);
       
       
       //Actual property which would be used.
       //Actual property which would be used.
       Properties queueConfProps = new Properties();
       Properties queueConfProps = new Properties();
-      queueConfProps.put("mapred.queue.default.acl-submit-job", " ");
+      queueConfProps.put(QueueManager.toFullPropertyName(
+          "default", submitAcl), " ");
       //Writing out the queue configuration file.
       //Writing out the queue configuration file.
       UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
       UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
       
       
       //Create a new configuration to be used with QueueManager
       //Create a new configuration to be used with QueueManager
       JobConf conf = new JobConf();
       JobConf conf = new JobConf();
       QueueManager queueManager = new QueueManager(conf);
       QueueManager queueManager = new QueueManager(conf);
-      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      UserGroupInformation ugi = UserGroupInformation.
+          createUserForTesting("user1", new String [] {"group1"});
+
       //Job Submission should fail because ugi to be used is set to blank.
       //Job Submission should fail because ugi to be used is set to blank.
       assertFalse("User Job Submission Succeeded before refresh.",
       assertFalse("User Job Submission Succeeded before refresh.",
           queueManager.hasAccess("default", QueueManager.QueueOperation.
           queueManager.hasAccess("default", QueueManager.QueueOperation.
@@ -345,10 +398,16 @@ public class TestQueueManager extends TestCase {
           queueManager.hasAccess("q2", QueueManager.QueueOperation.
           queueManager.hasAccess("q2", QueueManager.QueueOperation.
               SUBMIT_JOB, alternateUgi));
               SUBMIT_JOB, alternateUgi));
       
       
-      //Set acl for the current user.
-      queueConfProps.put("mapred.queue.default.acl-submit-job", ugi.getUserName());
-      queueConfProps.put("mapred.queue.q1.acl-submit-job", ugi.getUserName());
-      queueConfProps.put("mapred.queue.q2.acl-submit-job", ugi.getUserName());
+      //Set acl for user1.
+      queueConfProps.put(QueueManager.toFullPropertyName(
+          "default", submitAcl),
+    		  ugi.getShortUserName());
+      queueConfProps.put(QueueManager.toFullPropertyName(
+          "q1", submitAcl),
+    		  ugi.getShortUserName());
+      queueConfProps.put(QueueManager.toFullPropertyName(
+          "q2", submitAcl),
+    		  ugi.getShortUserName());
       //write out queue-acls.xml.
       //write out queue-acls.xml.
       UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
       UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
       //refresh configuration
       //refresh configuration
@@ -370,8 +429,10 @@ public class TestQueueManager extends TestCase {
       queueConfigFile.delete();
       queueConfigFile.delete();
       
       
       //rewrite the mapred-site.xml
       //rewrite the mapred-site.xml
-      hadoopConfProps.put("mapred.acls.enabled", "true");
-      hadoopConfProps.put("mapred.queue.default.acl-submit-job", ugi.getUserName());
+      hadoopConfProps.put(JobConf.MR_ACLS_ENABLED, "true");
+      hadoopConfProps.put(QueueManager.toFullPropertyName(
+          "default", submitAcl),
+          ugi.getShortUserName());
       UtilsForTests.setUpConfigFile(hadoopConfProps, hadoopConfigFile);
       UtilsForTests.setUpConfigFile(hadoopConfProps, hadoopConfigFile);
       queueManager.refreshAcls(conf);
       queueManager.refreshAcls(conf);
       assertTrue("User Job Submission failed after refresh and no queue acls file.",
       assertTrue("User Job Submission failed after refresh and no queue acls file.",
@@ -397,15 +458,21 @@ public class TestQueueManager extends TestCase {
       // queue properties with which the cluster is started.
       // queue properties with which the cluster is started.
       Properties hadoopConfProps = new Properties();
       Properties hadoopConfProps = new Properties();
       hadoopConfProps.put("mapred.queue.names", "default,q1,q2");
       hadoopConfProps.put("mapred.queue.names", "default,q1,q2");
-      hadoopConfProps.put("mapred.acls.enabled", "true");
+      hadoopConfProps.put(JobConf.MR_ACLS_ENABLED, "true");
       UtilsForTests.setUpConfigFile(hadoopConfProps, hadoopConfigFile);
       UtilsForTests.setUpConfigFile(hadoopConfProps, hadoopConfigFile);
       
       
       //properties for mapred-queue-acls.xml
       //properties for mapred-queue-acls.xml
       Properties queueConfProps = new Properties();
       Properties queueConfProps = new Properties();
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-      queueConfProps.put("mapred.queue.default.acl-submit-job", ugi.getUserName());
-      queueConfProps.put("mapred.queue.q1.acl-submit-job", ugi.getUserName());
-      queueConfProps.put("mapred.queue.q2.acl-submit-job", ugi.getUserName());
+      queueConfProps.put(QueueManager.toFullPropertyName(
+          "default", submitAcl),
+          ugi.getShortUserName());
+      queueConfProps.put(QueueManager.toFullPropertyName(
+          "q1", submitAcl),
+          ugi.getShortUserName());
+      queueConfProps.put(QueueManager.toFullPropertyName(
+          "q2", submitAcl),
+          ugi.getShortUserName());
       UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
       UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
       
       
       Configuration conf = new JobConf();
       Configuration conf = new JobConf();
@@ -457,7 +524,7 @@ public class TestQueueManager extends TestCase {
   
   
   private JobConf setupConf(String aclName, String aclValue) {
   private JobConf setupConf(String aclName, String aclValue) {
     JobConf conf = new JobConf();
     JobConf conf = new JobConf();
-    conf.setBoolean("mapred.acls.enabled", true);
+    conf.setBoolean(JobConf.MR_ACLS_ENABLED, true);
     conf.set(aclName, aclValue);
     conf.set(aclName, aclValue);
     return conf;
     return conf;
   }
   }
@@ -470,21 +537,30 @@ public class TestQueueManager extends TestCase {
     }
     }
   }
   }
   
   
-  private void verifyJobSubmission(JobConf conf, boolean shouldSucceed) 
-                                     throws IOException, InterruptedException {
-    verifyJobSubmission(conf, shouldSucceed, "default");
+  /**
+   *  Verify job submission as given user to the default queue
+   */
+  private void verifyJobSubmissionToDefaultQueue(JobConf conf, boolean shouldSucceed,
+		  String userInfo) throws IOException, InterruptedException {
+    verifyJobSubmission(conf, shouldSucceed, userInfo, "default");
   }
   }
 
 
+  /**
+   * Verify job submission as given user to the given queue
+   */
   private void verifyJobSubmission(JobConf conf, boolean shouldSucceed, 
   private void verifyJobSubmission(JobConf conf, boolean shouldSucceed, 
-      String queue) throws IOException, InterruptedException {
+      String userInfo, String queue) throws IOException, InterruptedException {
     setUpCluster(conf);
     setUpCluster(conf);
     try {
     try {
-      runAndVerifySubmission(conf, shouldSucceed, queue, null);
+      runAndVerifySubmission(conf, shouldSucceed, queue, userInfo);
     } finally {
     } finally {
       tearDownCluster();
       tearDownCluster();
     }
     }
   }
   }
 
 
+  /**
+   * Verify if submission of job to the given queue will succeed or not
+   */
   private void runAndVerifySubmission(JobConf conf, boolean shouldSucceed,
   private void runAndVerifySubmission(JobConf conf, boolean shouldSucceed,
       String queue, String userInfo)
       String queue, String userInfo)
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
@@ -519,8 +595,16 @@ public class TestQueueManager extends TestCase {
     }
     }
 }
 }
 
 
-  private void verifyJobKill(JobConf conf, boolean shouldSucceed) 
-                                      throws IOException, InterruptedException {
+  /**
+   * Submit job as current user and kill the job as user of ugi.
+   * @param ugi {@link UserGroupInformation} of user who tries to kill the job
+   * @param conf JobConf for the job
+   * @param shouldSucceed Should the killing of job be succeeded ?
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void verifyJobKill(UserGroupInformation ugi, JobConf conf,
+		  boolean shouldSucceed) throws IOException, InterruptedException {
     setUpCluster(conf);
     setUpCluster(conf);
     try {
     try {
       RunningJob rjob = submitSleepJob(1, 1, 1000, 1000, false);
       RunningJob rjob = submitSleepJob(1, 1, 1000, 1000, false);
@@ -532,7 +616,20 @@ public class TestQueueManager extends TestCase {
           break;
           break;
         }
         }
       }
       }
-      rjob.killJob();
+      conf.set("mapred.job.tracker", "localhost:"
+              + miniMRCluster.getJobTrackerPort());
+      final String jobId = rjob.getJobID();
+      ugi.doAs(new PrivilegedExceptionAction<Object>() {
+
+        @Override
+        public Object run() throws Exception {
+          RunningJob runningJob =
+        	  new JobClient(miniMRCluster.createJobConf()).getJob(jobId);
+          runningJob.killJob();
+          return null;
+        }
+      });
+
       while(rjob.cleanupProgress() == 0.0f) {
       while(rjob.cleanupProgress() == 0.0f) {
         try {
         try {
           Thread.sleep(10);  
           Thread.sleep(10);  
@@ -549,10 +646,9 @@ public class TestQueueManager extends TestCase {
       if (shouldSucceed) {
       if (shouldSucceed) {
         throw ioe;
         throw ioe;
       } else {
       } else {
-        LOG.info("exception while submitting job: " + ioe.getMessage());
+        LOG.info("exception while submitting/killing job: " + ioe.getMessage());
         assertTrue(ioe.getMessage().
         assertTrue(ioe.getMessage().
-                        contains("cannot perform operation " +
-                                    "ADMINISTER_JOBS on queue default"));
+            contains(" cannot perform operation MODIFY_JOB on "));
       }
       }
     } finally {
     } finally {
       tearDownCluster();
       tearDownCluster();
@@ -584,7 +680,7 @@ public class TestQueueManager extends TestCase {
           throw ioe;
           throw ioe;
         }
         }
         //verify it fails
         //verify it fails
-        LOG.info("exception while submitting job: " + ioe.getMessage());
+        LOG.info("exception while killing job: " + ioe.getMessage());
         assertTrue(ioe.getMessage().
         assertTrue(ioe.getMessage().
                         contains("cannot perform operation " +
                         contains("cannot perform operation " +
                                     "ADMINISTER_JOBS on queue default"));
                                     "ADMINISTER_JOBS on queue default"));
@@ -602,32 +698,51 @@ public class TestQueueManager extends TestCase {
     }
     }
   }
   }
   
   
-  private void verifyJobPriorityChangeAsOtherUser(JobConf conf, 
-                          boolean shouldSucceed, String otherUserInfo)
-                            throws IOException, InterruptedException {
+  /**
+   * Submit job as current user and try to change priority of that job as
+   * another user.
+   * @param otherUGI user who will try to change priority of job
+   * @param conf jobConf for the job
+   * @param shouldSucceed Should the changing of priority of job be succeeded ?
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void verifyJobPriorityChangeAsOtherUser(UserGroupInformation otherUGI,
+      JobConf conf, final boolean shouldSucceed)
+      throws IOException, InterruptedException {
     setUpCluster(conf);
     setUpCluster(conf);
     try {
     try {
-      // submit job as another user.
-      String userInfo = otherUserInfo;
-      RunningJob rjob = submitSleepJob(1, 1, 1000, 1000, false, userInfo);
+      // submit job as current user.
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      String[] groups = ugi.getGroupNames();
+      String userInfo = ugi.getShortUserName() + "," +
+                        groups[groups.length - 1];
+      final RunningJob rjob = submitSleepJob(1, 1, 1000, 1000, false, userInfo);
       assertFalse(rjob.isComplete());
       assertFalse(rjob.isComplete());
       
       
-      // try to change priority as self
-      try {
-        conf.set("mapred.job.tracker", "localhost:"
-            + miniMRCluster.getJobTrackerPort());
-        JobClient client = new JobClient(miniMRCluster.createJobConf());
-        client.getJob(rjob.getID()).setJobPriority("VERY_LOW");
-        if (!shouldSucceed) {
-          fail("changing priority should fail.");
+      conf.set("mapred.job.tracker", "localhost:"
+	            + miniMRCluster.getJobTrackerPort());
+      // try to change priority as other user
+      otherUGI.doAs(new PrivilegedExceptionAction<Object>() {
+
+        @Override
+        public Object run() throws Exception {
+      	  try {
+            JobClient client = new JobClient(miniMRCluster.createJobConf());
+            client.getJob(rjob.getID()).setJobPriority("VERY_LOW");
+             if (!shouldSucceed) {
+              fail("changing priority should fail.");
+             }
+          } catch (IOException ioe) {
+            //verify it fails
+            LOG.info("exception while changing priority of job: " +
+                     ioe.getMessage());
+            assertTrue(ioe.getMessage().
+                contains(" cannot perform operation MODIFY_JOB on "));
+          }
+          return null;
         }
         }
-      } catch (IOException ioe) {
-        //verify it fails
-        LOG.info("exception while submitting job: " + ioe.getMessage());
-        assertTrue(ioe.getMessage().
-                        contains("cannot perform operation " +
-                                    "ADMINISTER_JOBS on queue default"));
-      }
+      });
       //wait for job to complete on its own
       //wait for job to complete on its own
       while (!rjob.isComplete()) {
       while (!rjob.isComplete()) {
         try {
         try {

+ 5 - 3
src/test/org/apache/hadoop/mapred/TestRecoveryManager.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapred.MiniMRCluster.JobTrackerRunner;
 import org.apache.hadoop.mapred.MiniMRCluster.JobTrackerRunner;
+import org.apache.hadoop.mapred.QueueManager.QueueOperation;
 import org.apache.hadoop.mapred.TestJobInProgressListener.MyScheduler;
 import org.apache.hadoop.mapred.TestJobInProgressListener.MyScheduler;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.*;
 import org.junit.*;
@@ -233,10 +234,11 @@ public class TestRecoveryManager extends TestCase {
                                       true);
                                       true);
     mr.getJobTrackerConf().setInt("mapred.jobtracker.maxtasks.per.job", 25);
     mr.getJobTrackerConf().setInt("mapred.jobtracker.maxtasks.per.job", 25);
     
     
-    mr.getJobTrackerConf().setBoolean("mapred.acls.enabled" , true);
+    mr.getJobTrackerConf().setBoolean(JobConf.MR_ACLS_ENABLED, true);
     UserGroupInformation ugi = UserGroupInformation.getLoginUser();
     UserGroupInformation ugi = UserGroupInformation.getLoginUser();
-    mr.getJobTrackerConf().set("mapred.queue.default.acl-submit-job", 
-                               ugi.getUserName());
+    mr.getJobTrackerConf().set(QueueManager.toFullPropertyName(
+        "default", QueueOperation.SUBMIT_JOB.getAclName()), 
+        ugi.getUserName());
 
 
     // start the jobtracker
     // start the jobtracker
     LOG.info("Starting jobtracker");
     LOG.info("Starting jobtracker");

+ 1 - 0
src/test/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java

@@ -116,6 +116,7 @@ public class TestTaskTrackerLocalization extends TestCase {
       localDirs[i] = new File(ROOT_MAPRED_LOCAL_DIR, "0_" + i).getPath();
       localDirs[i] = new File(ROOT_MAPRED_LOCAL_DIR, "0_" + i).getPath();
     }
     }
     trackerFConf.setStrings("mapred.local.dir", localDirs);
     trackerFConf.setStrings("mapred.local.dir", localDirs);
+    trackerFConf.setBoolean(JobConf.MR_ACLS_ENABLED, true);
 
 
     // Create the job configuration file. Same as trackerConf in this test.
     // Create the job configuration file. Same as trackerConf in this test.
     jobConf = new JobConf(trackerFConf);
     jobConf = new JobConf(trackerFConf);

+ 69 - 43
src/test/org/apache/hadoop/mapred/TestWebUIAuthorization.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.http.TestHttpServer.DummyFilterInitializer;
 import org.apache.hadoop.http.TestHttpServer.DummyFilterInitializer;
 import org.apache.hadoop.mapred.JobHistory.Keys;
 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.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.examples.SleepJob;
 import org.apache.hadoop.examples.SleepJob;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.Groups;
@@ -58,6 +59,8 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
   private static String mrOwner = null;
   private static String mrOwner = null;
   // member of supergroup
   // member of supergroup
   private static final String superGroupMember = "user2";
   private static final String superGroupMember = "user2";
+  // admin of "default" queue
+  private static final String qAdmin = "user3";
   // "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
@@ -107,42 +110,44 @@ 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 the job
-   * (3) user mentioned in job-view-acls should be able to view the job
-   * (4) user mentioned in job-modify-acls but not in job-view-acls
+   * (2) superGroupMember can view any job
+   * (3) mrOwner can view any job
+   * (4) user mentioned in job-view-acls should be able to view the
+   *     job irrespective of job-modify-acls.
+   * (5) user mentioned in job-modify-acls but not in job-view-acls
    *     cannot view the job
    *     cannot view the job
-   * (5) other unauthorized users cannot view the job
+   * (6) other unauthorized users cannot view the job
    */
    */
   private void validateViewJob(String url, String method)
   private void validateViewJob(String url, String method)
       throws IOException {
       throws IOException {
-    assertEquals("Incorrect return code for " + 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 " + superGroupMember,
-        HttpURLConnection.HTTP_OK, getHttpStatusCode(url, superGroupMember,
-            method));
-    assertEquals("Incorrect return code for " + mrOwner,
+    assertEquals("Incorrect return code for supergroup-member " +
+        superGroupMember, HttpURLConnection.HTTP_OK,
+        getHttpStatusCode(url, superGroupMember, method));
+    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 " + viewColleague,
-        HttpURLConnection.HTTP_OK, getHttpStatusCode(url, viewColleague,
-            method));
-    assertEquals("Incorrect return code for " + viewAndModifyColleague,
-        HttpURLConnection.HTTP_OK, getHttpStatusCode(url,
-            viewAndModifyColleague, method));
-    assertEquals("Incorrect return code for " + modifyColleague,
-        HttpURLConnection.HTTP_UNAUTHORIZED, getHttpStatusCode(url,
-            modifyColleague, method));
-    assertEquals("Incorrect return code for " + unauthorizedUser,
-        HttpURLConnection.HTTP_UNAUTHORIZED, getHttpStatusCode(url,
-            unauthorizedUser, method));
+    assertEquals("Incorrect return code for user in job-view-acl " +
+        viewColleague, HttpURLConnection.HTTP_OK,
+        getHttpStatusCode(url, viewColleague, method));
+    assertEquals("Incorrect return code for user in job-view-acl and " +
+        "job-modify-acl " + viewAndModifyColleague, HttpURLConnection.HTTP_OK,
+        getHttpStatusCode(url, viewAndModifyColleague, method));
+    assertEquals("Incorrect return code for user in job-modify-acl " +
+        modifyColleague, HttpURLConnection.HTTP_UNAUTHORIZED,
+        getHttpStatusCode(url, modifyColleague, method));
+    assertEquals("Incorrect return code for unauthorizedUser " +
+        unauthorizedUser, HttpURLConnection.HTTP_UNAUTHORIZED,
+        getHttpStatusCode(url, unauthorizedUser, method));
   }
   }
 
 
   /**
   /**
    * 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 and superGroupMember can modify the job. But we are not
-   *     validating this in this method. Let the caller explicitly validate
-   *     this, if needed.
+   * (1) jobSubmitter, mrOwner, qAdmin and superGroupMember can modify the job.
+   *     But we are not validating this in this method. Let the caller
+   *     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
    *     modify the job
    *     modify the job
    * (3) user mentioned in job-modify-acls (irrespective of job-view-acls)
    * (3) user mentioned in job-modify-acls (irrespective of job-view-acls)
@@ -256,8 +261,11 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     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());
-    props.setProperty(JobConf.JOB_LEVEL_AUTHORIZATION_ENABLING_FLAG,
-        String.valueOf(true));
+
+    props.setProperty(JobConf.MR_ACLS_ENABLED, String.valueOf(true));
+    props.setProperty(QueueManager.toFullPropertyName(
+        "default", QueueOperation.ADMINISTER_JOBS.getAclName()), qAdmin);
+
     props.setProperty("dfs.permissions", "false");
     props.setProperty("dfs.permissions", "false");
 
 
     // Let us have history files on HDFS
     // Let us have history files on HDFS
@@ -272,9 +280,11 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     MyGroupsProvider.mapping.put(unauthorizedUser, Arrays.asList("evilSociety"));
     MyGroupsProvider.mapping.put(unauthorizedUser, Arrays.asList("evilSociety"));
     MyGroupsProvider.mapping.put(superGroupMember, Arrays.asList("superGroup"));
     MyGroupsProvider.mapping.put(superGroupMember, Arrays.asList("superGroup"));
     MyGroupsProvider.mapping.put(viewAndModifyColleague, Arrays.asList("group3"));
     MyGroupsProvider.mapping.put(viewAndModifyColleague, Arrays.asList("group3"));
+    MyGroupsProvider.mapping.put(qAdmin, Arrays.asList("group4"));
+
     mrOwner = UserGroupInformation.getCurrentUser().getShortUserName();
     mrOwner = UserGroupInformation.getCurrentUser().getShortUserName();
     MyGroupsProvider.mapping.put(mrOwner, Arrays.asList(
     MyGroupsProvider.mapping.put(mrOwner, Arrays.asList(
-        new String[] { "group4", "group5" }));
+        new String[] { "group5", "group6" }));
 
 
     startCluster(true, props);
     startCluster(true, props);
     MiniMRCluster cluster = getMRCluster();
     MiniMRCluster cluster = getMRCluster();
@@ -410,11 +420,13 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
    * (1) jobSubmitter, mrOwner and superGroupMember can do both view and modify
    * (1) jobSubmitter, mrOwner and superGroupMember 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 but not in job-modify-acls cannot
+   * (2) user mentioned in job-view-acls and job-modify-acls can do this
+   * (3) user mentioned in job-view-acls but not in job-modify-acls cannot
    *     do this
    *     do this
-   * (3) user mentioned in job-modify-acls but not in job-view-acls cannot
+   * (4) user mentioned in job-modify-acls but not in job-view-acls cannot
    *     do this
    *     do this
-   * (4) other unauthorized users cannot do this
+   * (5) qAdmin cannot do this because he doesn't have view access to the job
+   * (6) other unauthorized users cannot do this
    *
    *
    * @throws Exception
    * @throws Exception
    */
    */
@@ -441,11 +453,16 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
           getHttpStatusCode(url, unauthorizedUser, "POST"));
           getHttpStatusCode(url, unauthorizedUser, "POST"));
       assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
       assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
           getHttpStatusCode(url, modifyColleague, "POST"));
           getHttpStatusCode(url, modifyColleague, "POST"));
+      // As qAdmin doesn't have view access to job, he cannot kill the job
+      // from jobdetails web page. But qAdmin can kill job from jobtracker page.
+      assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
+          getHttpStatusCode(url, qAdmin, "POST"));
+
       assertEquals(HttpURLConnection.HTTP_OK,
       assertEquals(HttpURLConnection.HTTP_OK,
           getHttpStatusCode(url, viewAndModifyColleague, "POST"));
           getHttpStatusCode(url, viewAndModifyColleague, "POST"));
       waitForKillJobToFinish(job);
       waitForKillJobToFinish(job);
-      assertTrue("killJob failed for a job for which user has "
-          + "job-modify permission", job.isComplete());
+      assertTrue("killJob using jobdetails.jsp failed for a job for which "
+          + "user has job-view and job-modify permissions", job.isComplete());
     } finally {
     } finally {
       if (!job.isComplete()) {
       if (!job.isComplete()) {
         LOG.info("Killing job " + jobid + " from finally block");
         LOG.info("Killing job " + jobid + " from finally block");
@@ -455,8 +472,8 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
       }
       }
     }
     }
 
 
-    // check if jobSubmitter, mrOwner and superGroupMember can do killJob
-    // using jobdetails.jsp url
+    // check if jobSubmitter, mrOwner superGroupMember can do
+    // 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,
@@ -532,9 +549,10 @@ 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). So only
-    // based on the config JobContext.JOB_ACL_MODIFY_JOB being set here,
-    // killJob on each of the jobs will be succeeded.
+    // mrOwner and none of these users is a member of superGroup and none of
+    // 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
+    // job-submitter, killJob on each of the jobs will be succeeded.
 
 
     // start 1st job.
     // start 1st job.
     // Out of these 4 users, only jobSubmitter can do killJob on 1st job
     // Out of these 4 users, only jobSubmitter can do killJob on 1st job
@@ -603,8 +621,11 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     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());
-    props.setProperty(
-       JobConf.JOB_LEVEL_AUTHORIZATION_ENABLING_FLAG, String.valueOf(true));
+
+    props.setProperty(JobConf.MR_ACLS_ENABLED, String.valueOf(true));
+    props.setProperty(QueueManager.toFullPropertyName(
+        "default", QueueOperation.ADMINISTER_JOBS.getAclName()), qAdmin);
+
     props.setProperty("dfs.permissions", "false");
     props.setProperty("dfs.permissions", "false");
     // let us have enough map slots so that there won't be waiting for slots
     // let us have enough map slots so that there won't be waiting for slots
     props.setProperty("mapred.tasktracker.map.tasks.maximum", "6");
     props.setProperty("mapred.tasktracker.map.tasks.maximum", "6");
@@ -618,10 +639,11 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     MyGroupsProvider.mapping.put(unauthorizedUser, Arrays.asList("evilSociety"));
     MyGroupsProvider.mapping.put(unauthorizedUser, Arrays.asList("evilSociety"));
     MyGroupsProvider.mapping.put(superGroupMember, Arrays.asList("superGroup"));
     MyGroupsProvider.mapping.put(superGroupMember, Arrays.asList("superGroup"));
     MyGroupsProvider.mapping.put(viewAndModifyColleague, Arrays.asList("group3"));
     MyGroupsProvider.mapping.put(viewAndModifyColleague, Arrays.asList("group3"));
+    MyGroupsProvider.mapping.put(qAdmin, Arrays.asList("group4"));
 
 
     mrOwner = UserGroupInformation.getCurrentUser().getShortUserName();
     mrOwner = UserGroupInformation.getCurrentUser().getShortUserName();
     MyGroupsProvider.mapping.put(mrOwner, Arrays.asList(
     MyGroupsProvider.mapping.put(mrOwner, Arrays.asList(
-        new String[] { "group4", "group5" }));
+        new String[] { "group5", "group6" }));
 
 
     startCluster(true, props);
     startCluster(true, props);
     MiniMRCluster cluster = getMRCluster();
     MiniMRCluster cluster = getMRCluster();
@@ -676,8 +698,8 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
         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, superGroupMember);
+    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
     // if all the jobs which can be killed by user are actually the ones that
     // if all the jobs which can be killed by user are actually the ones that
@@ -725,13 +747,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 and superGroupMember are not validated for
+    // jobSubmitter, mrOwner, qAdmin and superGroupMember 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, superGroupMember, "GET"));
+    assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
+        jobTrackerJSPSetJobPriorityAction, qAdmin, "GET"));
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
     assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(
         jobTrackerJSPSetJobPriorityAction, mrOwner, "GET"));
         jobTrackerJSPSetJobPriorityAction, mrOwner, "GET"));
   }
   }
@@ -807,6 +831,8 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
         getHttpStatusCode(jobTrackerJSP, viewAndModifyColleague, "GET"));
         getHttpStatusCode(jobTrackerJSP, viewAndModifyColleague, "GET"));
     assertEquals(HttpURLConnection.HTTP_OK,
     assertEquals(HttpURLConnection.HTTP_OK,
         getHttpStatusCode(jobTrackerJSP, mrOwner, "GET"));
         getHttpStatusCode(jobTrackerJSP, mrOwner, "GET"));
+    assertEquals(HttpURLConnection.HTTP_OK,
+        getHttpStatusCode(jobTrackerJSP, qAdmin, "GET"));
     assertEquals(HttpURLConnection.HTTP_OK,
     assertEquals(HttpURLConnection.HTTP_OK,
         getHttpStatusCode(jobTrackerJSP, superGroupMember, "GET"));
         getHttpStatusCode(jobTrackerJSP, superGroupMember, "GET"));
   }
   }