You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 05:18:28 UTC

svn commit: r1077471 [2/2] - in /hadoop/common/branches/branch-0.20-security-patches: conf/ src/c++/task-controller/ src/c++/task-controller/tests/ src/core/org/apache/hadoop/security/authorize/ src/docs/src/documentation/content/xdocs/ src/mapred/ src...

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java?rev=1077471&r1=1077470&r2=1077471&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java Fri Mar  4 04:18:27 2011
@@ -38,15 +38,15 @@ import org.apache.hadoop.examples.SleepJ
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.mapred.QueueManager.QueueOperation;
+import org.apache.hadoop.mapred.QueueManager.QueueACL;
 import org.apache.hadoop.security.UserGroupInformation;
 
 public class TestQueueManager extends TestCase {
 
   private static final Log LOG = LogFactory.getLog(TestQueueManager.class);
 
-  String submitAcl = QueueOperation.SUBMIT_JOB.getAclName();
-  String adminAcl  = QueueOperation.ADMINISTER_JOBS.getAclName();
+  String submitAcl = QueueACL.SUBMIT_JOB.getAclName();
+  String adminAcl  = QueueACL.ADMINISTER_JOBS.getAclName();
 
   private MiniDFSCluster miniDFSCluster;
   private MiniMRCluster miniMRCluster;
@@ -389,62 +389,49 @@ public class TestQueueManager extends Te
 
       //Job Submission should fail because ugi to be used is set to blank.
       assertFalse("User Job Submission Succeeded before refresh.",
-          queueManager.hasAccess("default", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("default", QueueACL.SUBMIT_JOB, ugi));
       assertFalse("User Job Submission Succeeded before refresh.",
-          queueManager.hasAccess("q1", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q1", QueueACL.SUBMIT_JOB, ugi));
       assertFalse("User Job Submission Succeeded before refresh.",
-          queueManager.hasAccess("q2", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q2", QueueACL.SUBMIT_JOB, ugi));
       
       //Test job submission as alternate user.
       UserGroupInformation alternateUgi = 
         UserGroupInformation.createUserForTesting("u1", new String[]{"user"});
       assertTrue("Alternate User Job Submission failed before refresh.",
-          queueManager.hasAccess("q2", QueueManager.QueueOperation.
-              SUBMIT_JOB, alternateUgi));
+          queueManager.hasAccess("q2", QueueACL.SUBMIT_JOB, alternateUgi));
       
       //Set acl for user1.
       queueConfProps.put(QueueManager.toFullPropertyName(
-          "default", submitAcl),
-    		  ugi.getShortUserName());
+          "default", submitAcl), ugi.getShortUserName());
       queueConfProps.put(QueueManager.toFullPropertyName(
-          "q1", submitAcl),
-    		  ugi.getShortUserName());
+          "q1", submitAcl), ugi.getShortUserName());
       queueConfProps.put(QueueManager.toFullPropertyName(
-          "q2", submitAcl),
-    		  ugi.getShortUserName());
+          "q2", submitAcl), ugi.getShortUserName());
       //write out queue-acls.xml.
       UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
       //refresh configuration
       queueManager.refreshAcls(conf);
       //Submission should succeed
       assertTrue("User Job Submission failed after refresh.",
-          queueManager.hasAccess("default", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("default", QueueACL.SUBMIT_JOB, ugi));
       assertTrue("User Job Submission failed after refresh.",
-          queueManager.hasAccess("q1", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q1", QueueACL.SUBMIT_JOB, ugi));
       assertTrue("User Job Submission failed after refresh.",
-          queueManager.hasAccess("q2", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q2", QueueACL.SUBMIT_JOB, ugi));
       assertFalse("Alternate User Job Submission succeeded after refresh.",
-          queueManager.hasAccess("q2", QueueManager.QueueOperation.
-              SUBMIT_JOB, alternateUgi));
+          queueManager.hasAccess("q2", QueueACL.SUBMIT_JOB, alternateUgi));
       //delete the ACL file.
       queueConfigFile.delete();
       
       //rewrite the mapred-site.xml
       hadoopConfProps.put(JobConf.MR_ACLS_ENABLED, "true");
       hadoopConfProps.put(QueueManager.toFullPropertyName(
-          "q1", submitAcl),
-          ugi.getShortUserName());
+          "q1", submitAcl), ugi.getShortUserName());
       UtilsForTests.setUpConfigFile(hadoopConfProps, hadoopConfigFile);
       queueManager.refreshAcls(conf);
       assertTrue("User Job Submission allowed after refresh and no queue acls file.",
-          queueManager.hasAccess("q1", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q1", QueueACL.SUBMIT_JOB, ugi));
     } finally{
       if(queueConfigFile.exists()) {
         queueConfigFile.delete();
@@ -472,28 +459,22 @@ public class TestQueueManager extends Te
       Properties queueConfProps = new Properties();
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
       queueConfProps.put(QueueManager.toFullPropertyName(
-          "default", submitAcl),
-          ugi.getShortUserName());
+          "default", submitAcl), ugi.getShortUserName());
       queueConfProps.put(QueueManager.toFullPropertyName(
-          "q1", submitAcl),
-          ugi.getShortUserName());
+          "q1", submitAcl), ugi.getShortUserName());
       queueConfProps.put(QueueManager.toFullPropertyName(
-          "q2", submitAcl),
-          ugi.getShortUserName());
+          "q2", submitAcl), ugi.getShortUserName());
       UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
       
       Configuration conf = new JobConf();
       QueueManager queueManager = new QueueManager(conf);
       //Testing access to queue.
       assertTrue("User Job Submission failed.",
-          queueManager.hasAccess("default", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("default", QueueACL.SUBMIT_JOB, ugi));
       assertTrue("User Job Submission failed.",
-          queueManager.hasAccess("q1", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q1", QueueACL.SUBMIT_JOB, ugi));
       assertTrue("User Job Submission failed.",
-          queueManager.hasAccess("q2", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q2", QueueACL.SUBMIT_JOB, ugi));
       
       //Write out a new incomplete invalid configuration file.
       PrintWriter writer = new PrintWriter(new FileOutputStream(queueConfigFile));
@@ -509,14 +490,11 @@ public class TestQueueManager extends Te
       } catch (Exception e) {
       }
       assertTrue("User Job Submission failed after invalid conf file refresh.",
-          queueManager.hasAccess("default", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("default", QueueACL.SUBMIT_JOB, ugi));
       assertTrue("User Job Submission failed after invalid conf file refresh.",
-          queueManager.hasAccess("q1", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q1", QueueACL.SUBMIT_JOB, ugi));
       assertTrue("User Job Submission failed after invalid conf file refresh.",
-          queueManager.hasAccess("q2", QueueManager.QueueOperation.
-              SUBMIT_JOB, ugi));
+          queueManager.hasAccess("q2", QueueACL.SUBMIT_JOB, ugi));
     } finally {
       //Cleanup the configuration files in all cases
       if(hadoopConfigFile.exists()) {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestRecoveryManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestRecoveryManager.java?rev=1077471&r1=1077470&r2=1077471&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestRecoveryManager.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestRecoveryManager.java Fri Mar  4 04:18:27 2011
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapred.MiniMRCluster.JobTrackerRunner;
-import org.apache.hadoop.mapred.QueueManager.QueueOperation;
+import org.apache.hadoop.mapred.QueueManager.QueueACL;
 import org.apache.hadoop.mapred.TestJobInProgressListener.MyScheduler;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.*;
@@ -237,8 +237,7 @@ public class TestRecoveryManager extends
     mr.getJobTrackerConf().setBoolean(JobConf.MR_ACLS_ENABLED, true);
     UserGroupInformation ugi = UserGroupInformation.getLoginUser();
     mr.getJobTrackerConf().set(QueueManager.toFullPropertyName(
-        "default", QueueOperation.SUBMIT_JOB.getAclName()), 
-        ugi.getUserName());
+        "default", QueueACL.SUBMIT_JOB.getAclName()), ugi.getUserName());
 
     // start the jobtracker
     LOG.info("Starting jobtracker");

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java?rev=1077471&r1=1077470&r2=1077471&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java Fri Mar  4 04:18:27 2011
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JvmManager.JvmEnv;
+import org.apache.hadoop.mapred.QueueManager.QueueACL;
 import org.apache.hadoop.mapred.TaskController.JobInitializationContext;
 import org.apache.hadoop.mapred.TaskController.TaskControllerContext;
 import org.apache.hadoop.mapred.TaskTracker.RunningJob;
@@ -127,6 +128,14 @@ public class TestTaskTrackerLocalization
     jobConf.setInt("mapred.userlog.retain.hours", 0);
     jobConf.setUser(getJobOwner().getShortUserName());
 
+    // set job queue name in job conf
+    String queue = "default";
+    jobConf.setQueueName(queue);
+    // Set queue admins acl in job conf similar to what JobClient does
+    jobConf.set(QueueManager.toFullPropertyName(queue,
+        QueueACL.ADMINISTER_JOBS.getAclName()),
+        "qAdmin1,qAdmin2 qAdminsGroup1,qAdminsGroup2");
+
     String jtIdentifier = "200907202331";
     jobId = new JobID(jtIdentifier, 1);
 
@@ -468,6 +477,34 @@ public class TestTaskTrackerLocalization
         .exists());
     checkFilePermissions(jobLogDir.toString(), "drwx------", task.getUser(),
         taskTrackerUGI.getGroupNames()[0]);
+
+    // Make sure that the job ACLs file job-acls.xml exists in job userlog dir
+    File jobACLsFile = new File(jobLogDir, TaskTracker.jobACLsFile);
+    assertTrue("JobACLsFile is missing in the job userlog dir " + jobLogDir,
+        jobACLsFile.exists());
+
+    // With default task controller, the job-acls.xml file is owned by TT and
+    // permissions are 700
+    checkFilePermissions(jobACLsFile.getAbsolutePath(), "-rwx------",
+        taskTrackerUGI.getShortUserName(), taskTrackerUGI.getGroupNames()[0]);
+
+    validateJobACLsFileContent();
+  }
+
+  // Validate the contents of jobACLsFile ( i.e. user name, job-view-acl, queue
+  // name and queue-admins-acl ).
+  protected void validateJobACLsFileContent() {
+    JobConf jobACLsConf = TaskLogServlet.getConfFromJobACLsFile(jobId);
+    assertTrue(jobACLsConf.get("user.name").equals(
+        localizedJobConf.getUser()));
+    assertTrue(jobACLsConf.get(JobContext.JOB_ACL_VIEW_JOB).
+        equals(localizedJobConf.get(JobContext.JOB_ACL_VIEW_JOB)));
+    String queue = localizedJobConf.getQueueName();
+    assertTrue(queue.equalsIgnoreCase(jobACLsConf.getQueueName()));
+    String qACLName = QueueManager.toFullPropertyName(queue,
+        QueueACL.ADMINISTER_JOBS.getAclName());
+    assertTrue(jobACLsConf.get(qACLName).equals(
+        localizedJobConf.get(qACLName)));
   }
 
   /**
@@ -525,24 +562,6 @@ public class TestTaskTrackerLocalization
         + expectedStderr.toString() + " Observed : "
         + attemptLogFiles[1].toString(), expectedStderr.toString().equals(
         attemptLogFiles[1].toString()));
-
-    // Make sure that the job ACLs file exists in the task log dir
-    File jobACLsFile = new File(logDir, TaskRunner.jobACLsFile);
-    assertTrue("JobACLsFile is missing in the task log dir " + logDir,
-        jobACLsFile.exists());
-
-    // With default task controller, the job-acls file is owned by TT and
-    // permissions are 700
-    checkFilePermissions(jobACLsFile.getAbsolutePath(), "-rwx------",
-        taskTrackerUGI.getShortUserName(), taskTrackerUGI.getGroupNames()[0]);
-
-    // Validate the contents of jobACLsFile(both user name and job-view-acls)
-    Configuration jobACLsConf = TaskLogServlet.getConfFromJobACLsFile(
-        task.getTaskID(), task.isTaskCleanupTask());
-    assertTrue(jobACLsConf.get("user.name").equals(
-        localizedJobConf.getUser()));
-    assertTrue(jobACLsConf.get(JobContext.JOB_ACL_VIEW_JOB).
-        equals(localizedJobConf.get(JobContext.JOB_ACL_VIEW_JOB)));
   }
 
   /**

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestUserLogCleanup.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestUserLogCleanup.java?rev=1077471&r1=1077470&r2=1077471&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestUserLogCleanup.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestUserLogCleanup.java Fri Mar  4 04:18:27 2011
@@ -60,8 +60,9 @@ public class TestUserLogCleanup {
 
   private File localizeJob(JobID jobid) throws IOException {
     File jobUserlog = TaskLog.getJobDir(jobid);
+    JobConf conf = new JobConf();
     // localize job log directory
-    tt.initializeJobLogDir(jobid);
+    tt.initializeJobLogDir(jobid, conf);
     assertTrue(jobUserlog + " directory is not created.", jobUserlog.exists());
     return jobUserlog;
   }
@@ -75,6 +76,7 @@ public class TestUserLogCleanup {
   private void startTT(Configuration conf) throws IOException {
     myClock = new FakeClock(); // clock is reset.
     tt = new TaskTracker();
+    tt.setConf(new JobConf(conf));
     localizer = new Localizer(FileSystem.get(conf), conf
         .getStrings(JobConf.MAPRED_LOCAL_DIR_PROPERTY),
         new DefaultTaskController());

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestWebUIAuthorization.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestWebUIAuthorization.java?rev=1077471&r1=1077470&r2=1077471&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestWebUIAuthorization.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestWebUIAuthorization.java Fri Mar  4 04:18:27 2011
@@ -33,7 +33,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.http.TestHttpServer.DummyFilterInitializer;
 import org.apache.hadoop.mapred.JobHistory.Keys;
 import org.apache.hadoop.mapred.JobHistory.TaskAttempt;
-import org.apache.hadoop.mapred.QueueManager.QueueOperation;
+import org.apache.hadoop.mapred.QueueManager.QueueACL;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.examples.SleepJob;
@@ -121,11 +121,13 @@ public class TestWebUIAuthorization exte
    * (1) jobSubmitter can view the job
    * (2) mrAdmin 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
+   * (4) qAdmins of the queue to which job is submitted to can view any job in
+   *     that queue.
+   * (5) user mentioned in job-view-acl should be able to view the
+   *     job irrespective of job-modify-acl.
+   * (6) user mentioned in job-modify-acl but not in job-view-acl
    *     cannot view the job
-   * (6) other unauthorized users cannot view the job
+   * (7) other unauthorized users cannot view the job
    */
   private void validateViewJob(String url, String method)
       throws IOException {
@@ -140,6 +142,8 @@ public class TestWebUIAuthorization exte
         getHttpStatusCode(url, mrAdminGroupMember, method));
     assertEquals("Incorrect return code for MR-owner " + mrOwner,
         HttpURLConnection.HTTP_OK, getHttpStatusCode(url, mrOwner, method));
+    assertEquals("Incorrect return code for queue admin " + qAdmin,
+        HttpURLConnection.HTTP_OK, getHttpStatusCode(url, qAdmin, method));
     assertEquals("Incorrect return code for user in job-view-acl " +
         viewColleague, HttpURLConnection.HTTP_OK,
         getHttpStatusCode(url, viewColleague, method));
@@ -160,9 +164,9 @@ public class TestWebUIAuthorization exte
    * (1) jobSubmitter, mrOwner, qAdmin and mrAdmin 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-acl but not in job-modify-acl cannot
    *     modify the job
-   * (3) user mentioned in job-modify-acls (irrespective of job-view-acls)
+   * (3) user mentioned in job-modify-acl (irrespective of job-view-acl)
    *     can modify the job
    * (4) other unauthorized users cannot modify the job
    */
@@ -297,9 +301,9 @@ public class TestWebUIAuthorization exte
 
     props.setProperty(JobConf.MR_ACLS_ENABLED, String.valueOf(true));
     props.setProperty(QueueManager.toFullPropertyName(
-        "default", QueueOperation.ADMINISTER_JOBS.getAclName()), qAdmin);
+        "default", QueueACL.ADMINISTER_JOBS.getAclName()), qAdmin);
     props.setProperty(QueueManager.toFullPropertyName(
-        "default", QueueOperation.SUBMIT_JOB.getAclName()), jobSubmitter);
+        "default", QueueACL.SUBMIT_JOB.getAclName()), jobSubmitter);
 
     props.setProperty("dfs.permissions", "false");
 
@@ -383,7 +387,9 @@ public class TestWebUIAuthorization exte
     JobHistory.parseHistoryFromFS(historyFilePath.toString().substring(5),
         l, historyFilePath.getFileSystem(conf));
 
-    Map<String, org.apache.hadoop.mapred.JobHistory.Task> tipsMap = jobInfo.getAllTasks();
+    Map<String, org.apache.hadoop.mapred.JobHistory.Task> tipsMap =
+    	jobInfo.getAllTasks();
+
     for (String tip : tipsMap.keySet()) {
       // validate access of taskdetailshistory.jsp
       validateViewJob(jtURL + "/taskdetailshistory.jsp?logFile="
@@ -401,20 +407,37 @@ public class TestWebUIAuthorization exte
         String taskLogURL = TaskLogServlet.getTaskLogUrl("localhost",
             attemptsMap.get(attempt).get(Keys.HTTP_PORT), attempt.toString());
         validateViewJob(taskLogURL, "GET");
+      }
+    }
 
-        // delete job-acls.xml file from the task log dir of attempt and verify
-        // if unauthorized users can view task logs of attempt.
-        File attemptLogDir = TaskLog.getAttemptDir(TaskAttemptID
-            .forName(attempt), false);
-        Path jobACLsFilePath = new Path(attemptLogDir.toString(),
-            TaskRunner.jobACLsFile);
-        new File(jobACLsFilePath.toUri().getPath()).delete();
+    // For each tip, let us test the effect of deletion of job-acls.xml file and
+    // deletion of task log dir for each of the attempts of the tip.
+    
+    // delete job-acls.xml file from the job userlog dir and verify
+    // if unauthorized users can view task logs of each attempt.
+    Path jobACLsFilePath = new Path(TaskLog.getJobDir(jobid).toString(),
+        TaskTracker.jobACLsFile);
+    new File(jobACLsFilePath.toUri().getPath()).delete();
+
+    for (String tip : tipsMap.keySet()) {
+
+      Map<String, TaskAttempt> attemptsMap =
+        tipsMap.get(tip).getTaskAttempts();
+      for (String attempt : attemptsMap.keySet()) {
+
+        String taskLogURL = TaskLogServlet.getTaskLogUrl("localhost",
+            attemptsMap.get(attempt).get(Keys.HTTP_PORT), attempt.toString());
+
+        // unauthorized users can view task logs of each attempt because
+        // job-acls.xml file is deleted.
         assertEquals("Incorrect return code for " + unauthorizedUser,
             HttpURLConnection.HTTP_OK, getHttpStatusCode(taskLogURL,
                 unauthorizedUser, "GET"));
 
         // delete the whole task log dir of attempt and verify that we get
         // correct response code (i.e. HTTP_GONE) when task logs are accessed.
+        File attemptLogDir = TaskLog.getAttemptDir(TaskAttemptID
+            .forName(attempt), false);
         FileUtil.fullyDelete(attemptLogDir);
         assertEquals("Incorrect return code for " + jobSubmitter,
             HttpURLConnection.HTTP_GONE, getHttpStatusCode(taskLogURL,
@@ -477,10 +500,6 @@ public class TestWebUIAuthorization exte
           getHttpStatusCode(url, unauthorizedUser, "POST"));
       assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
           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,
           getHttpStatusCode(url, viewAndModifyColleague, "POST"));
@@ -499,13 +518,15 @@ public class TestWebUIAuthorization exte
     // check if jobSubmitter, mrOwner and mrAdmin can do
     // killJob using jobdetails.jsp url
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
-                                       jobSubmitter);
+                                      jobSubmitter);
+    confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
+                                      mrOwner);
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
-                                       mrOwner);
+                                      mrAdminUser);
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
-                                       mrAdminGroupMember);
+                                      mrAdminGroupMember);
     confirmJobDetailsJSPKillJobAsUser(cluster, conf, jtURL, jobTrackerJSP,
-        mrAdminUser);
+                                      qAdmin);
   }
 
   /**
@@ -647,9 +668,9 @@ public class TestWebUIAuthorization exte
 
     props.setProperty(JobConf.MR_ACLS_ENABLED, String.valueOf(true));
     props.setProperty(QueueManager.toFullPropertyName(
-        "default", QueueOperation.ADMINISTER_JOBS.getAclName()), qAdmin);
+        "default", QueueACL.ADMINISTER_JOBS.getAclName()), qAdmin);
     props.setProperty(QueueManager.toFullPropertyName(
-        "default", QueueOperation.SUBMIT_JOB.getAclName()),
+        "default", QueueACL.SUBMIT_JOB.getAclName()),
         jobSubmitter + "," + jobSubmitter1 + "," + jobSubmitter2 + "," +
         jobSubmitter3);
 
@@ -732,7 +753,6 @@ public class TestWebUIAuthorization exte
 
     startCluster(true, props);
     validateCommonServlets(getMRCluster());
-    stopCluster();
   }
 
   private void validateCommonServlets(MiniMRCluster cluster) throws IOException {