Ver Fonte

MAPREDUCE-3417. Fixed job-access-controls to work with MR AM and JobHistoryServer web-apps. Contributed by Jonathan Eagles.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1240428 13f79535-47bb-0310-9956-ffa450edef68
Vinod Kumar Vavilapalli há 13 anos atrás
pai
commit
fe1cf3b0ac

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -680,6 +680,9 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3760. Changed active nodes list to not contain unhealthy nodes
     on the webUI and metrics. (vinodkv)
 
+    MAPREDUCE-3417. Fixed job-access-controls to work with MR AM and
+    JobHistoryServer web-apps. (Jonathan Eagles via vinodkv)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 0 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -431,9 +431,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
   @Override
   public boolean checkAccess(UserGroupInformation callerUGI, 
       JobACL jobOperation) {
-    if (!UserGroupInformation.isSecurityEnabled()) {
-      return true;
-    }
     AccessControlList jobACL = jobACLs.get(jobOperation);
     return aclsManager.checkAccess(callerUGI, jobOperation, username, jobACL);
   }

+ 109 - 46
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java

@@ -95,7 +95,13 @@ public class AppController extends Controller implements AMParams {
    * Render the /job page
    */
   public void job() {
-    requireJob();
+    try {
+      requireJob();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     render(jobPage());
   }
 
@@ -110,7 +116,13 @@ public class AppController extends Controller implements AMParams {
    * Render the /jobcounters page
    */
   public void jobCounters() {
-    requireJob();
+    try {
+      requireJob();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     if (app.getJob() != null) {
       setTitle(join("Counters for ", $(JOB_ID)));
     }
@@ -121,7 +133,13 @@ public class AppController extends Controller implements AMParams {
    * Display a page showing a task's counters
    */
   public void taskCounters() {
-    requireTask();
+    try {
+      requireTask();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     if (app.getTask() != null) {
       setTitle(StringHelper.join("Counters for ", $(TASK_ID)));
     }
@@ -140,7 +158,13 @@ public class AppController extends Controller implements AMParams {
    * @throws IOException on any error.
    */
   public void singleJobCounter() throws IOException{
-    requireJob();
+    try {
+      requireJob();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     set(COUNTER_GROUP, URLDecoder.decode($(COUNTER_GROUP), "UTF-8"));
     set(COUNTER_NAME, URLDecoder.decode($(COUNTER_NAME), "UTF-8"));
     if (app.getJob() != null) {
@@ -155,7 +179,13 @@ public class AppController extends Controller implements AMParams {
    * @throws IOException on any error.
    */
   public void singleTaskCounter() throws IOException{
-    requireTask();
+    try {
+      requireTask();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     set(COUNTER_GROUP, URLDecoder.decode($(COUNTER_GROUP), "UTF-8"));
     set(COUNTER_NAME, URLDecoder.decode($(COUNTER_NAME), "UTF-8"));
     if (app.getTask() != null) {
@@ -176,7 +206,13 @@ public class AppController extends Controller implements AMParams {
    * Render the /tasks page
    */
   public void tasks() {
-    requireJob();
+    try {
+      requireJob();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     if (app.getJob() != null) {
       try {
         String tt = $(TASK_TYPE);
@@ -201,7 +237,13 @@ public class AppController extends Controller implements AMParams {
    * Render the /task page
    */
   public void task() {
-    requireTask();
+    try {
+      requireTask();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     if (app.getTask() != null) {
       setTitle(join("Attempts for ", $(TASK_ID)));
     }
@@ -219,7 +261,13 @@ public class AppController extends Controller implements AMParams {
    * Render the attempts page
    */
   public void attempts() {
-    requireJob();
+    try {
+      requireJob();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     if (app.getJob() != null) {
       try {
         String taskType = $(TASK_TYPE);
@@ -252,6 +300,13 @@ public class AppController extends Controller implements AMParams {
    */
   public void conf() {
     requireJob();
+    try {
+      requireJob();
+    }
+    catch (Exception e) {
+      renderText(e.getMessage());
+      return;
+    }
     render(confPage());
   }
 
@@ -280,41 +335,43 @@ public class AppController extends Controller implements AMParams {
   void accessDenied(String s) {
     setStatus(HttpServletResponse.SC_FORBIDDEN);
     setTitle(join("Access denied: ", s));
-    throw new RuntimeException("Access denied: " + s);
   }
 
   /**
    * check for job access.
    * @param job the job that is being accessed
+   * @return True if the requesting user has permission to view the job
    */
-  void checkAccess(Job job) {
+  boolean checkAccess(Job job) {
     UserGroupInformation callerUgi = UserGroupInformation.createRemoteUser(
         request().getRemoteUser());
-    if (!job.checkAccess(callerUgi, JobACL.VIEW_JOB)) {
-      accessDenied("User " + request().getRemoteUser() + " does not have " +
-          " permissions.");
-    }
+    return job.checkAccess(callerUgi, JobACL.VIEW_JOB);
   }
 
   /**
    * Ensure that a JOB_ID was passed into the page.
    */
   public void requireJob() {
-    try {
-      if ($(JOB_ID).isEmpty()) {
-        throw new RuntimeException("missing job ID");
-      }
-      JobId jobID = MRApps.toJobID($(JOB_ID));
-      app.setJob(app.context.getJob(jobID));
-      if (app.getJob() == null) {
-        notFound($(JOB_ID));
-      }
-      /* check for acl access */
-      Job job = app.context.getJob(jobID);
-      checkAccess(job);
-    } catch (Exception e) {
-      badRequest(e.getMessage() == null ? 
-          e.getClass().getName() : e.getMessage());
+    if ($(JOB_ID).isEmpty()) {
+      badRequest("missing job ID");
+      throw new RuntimeException("Bad Request: Missing job ID");
+    }
+
+    JobId jobID = MRApps.toJobID($(JOB_ID));
+    app.setJob(app.context.getJob(jobID));
+    if (app.getJob() == null) {
+      notFound($(JOB_ID));
+      throw new RuntimeException("Not Found: " + $(JOB_ID));
+    }
+
+    /* check for acl access */
+    Job job = app.context.getJob(jobID);
+    if (!checkAccess(job)) {
+      accessDenied("User " + request().getRemoteUser() + " does not have " +
+          " permission to view job " + $(JOB_ID));
+      throw new RuntimeException("Access denied: User " +
+          request().getRemoteUser() + " does not have permission to view job " +
+          $(JOB_ID));
     }
   }
 
@@ -322,24 +379,30 @@ public class AppController extends Controller implements AMParams {
    * Ensure that a TASK_ID was passed into the page.
    */
   public void requireTask() {
-    try {
-      if ($(TASK_ID).isEmpty()) {
-        throw new RuntimeException("missing task ID");
-      }
-      TaskId taskID = MRApps.toTaskID($(TASK_ID));
-      Job job = app.context.getJob(taskID.getJobId());
-      app.setJob(job);
-      if (app.getJob() == null) {
-        notFound(MRApps.toString(taskID.getJobId()));
-      } else {
-        app.setTask(app.getJob().getTask(taskID));
-        if (app.getTask() == null) {
-          notFound($(TASK_ID));
-        }
+    if ($(TASK_ID).isEmpty()) {
+      badRequest("missing task ID");
+      throw new RuntimeException("missing task ID");
+    }
+
+    TaskId taskID = MRApps.toTaskID($(TASK_ID));
+    Job job = app.context.getJob(taskID.getJobId());
+    app.setJob(job);
+    if (app.getJob() == null) {
+      notFound(MRApps.toString(taskID.getJobId()));
+      throw new RuntimeException("Not Found: " + $(JOB_ID));
+    } else {
+      app.setTask(app.getJob().getTask(taskID));
+      if (app.getTask() == null) {
+        notFound($(TASK_ID));
+        throw new RuntimeException("Not Found: " + $(TASK_ID));
       }
-      checkAccess(job);
-    } catch (Exception e) {
-      badRequest(e.getMessage());
+    }
+    if (!checkAccess(job)) {
+      accessDenied("User " + request().getRemoteUser() + " does not have " +
+          " permission to view job " + $(JOB_ID));
+      throw new RuntimeException("Access denied: User " +
+          request().getRemoteUser() + " does not have permission to view job " +
+          $(JOB_ID));
     }
   }
 }

+ 63 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java

@@ -28,6 +28,11 @@ import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.JobACL;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.TypeConverter;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.JobNoTasksCompletedTransition;
@@ -37,6 +42,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
 import org.apache.hadoop.mapreduce.v2.app.MRApp;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.junit.Test;
 import org.junit.Assert;
@@ -134,4 +140,61 @@ public class TestJobImpl {
     t.testCheckJobCompleteSuccess();
     t.testCheckJobCompleteSuccessFailed();
   }
+
+  @Test
+  public void testCheckAccess() {
+    // Create two unique users
+    String user1 = System.getProperty("user.name");
+    String user2 = user1 + "1234";
+    UserGroupInformation ugi1 = UserGroupInformation.createRemoteUser(user1);
+    UserGroupInformation ugi2 = UserGroupInformation.createRemoteUser(user2);
+
+    // Create the job
+    JobID jobID = JobID.forName("job_1234567890000_0001");
+    JobId jobId = TypeConverter.toYarn(jobID);
+
+    // Setup configuration access only to user1 (owner)
+    Configuration conf1 = new Configuration();
+    conf1.setBoolean(MRConfig.MR_ACLS_ENABLED, true);
+    conf1.set(MRJobConfig.JOB_ACL_VIEW_JOB, "");
+
+    // Verify access
+    JobImpl job1 = new JobImpl(jobId, null, conf1, null, null, null, null, null,
+        null, null, null, true, null, 0, null);
+    Assert.assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB));
+    Assert.assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB));
+
+    // Setup configuration access to the user1 (owner) and user2
+    Configuration conf2 = new Configuration();
+    conf2.setBoolean(MRConfig.MR_ACLS_ENABLED, true);
+    conf2.set(MRJobConfig.JOB_ACL_VIEW_JOB, user2);
+
+    // Verify access
+    JobImpl job2 = new JobImpl(jobId, null, conf2, null, null, null, null, null,
+        null, null, null, true, null, 0, null);
+    Assert.assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB));
+    Assert.assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB));
+
+    // Setup configuration access with security enabled and access to all
+    Configuration conf3 = new Configuration();
+    conf3.setBoolean(MRConfig.MR_ACLS_ENABLED, true);
+    conf3.set(MRJobConfig.JOB_ACL_VIEW_JOB, "*");
+
+    // Verify access
+    JobImpl job3 = new JobImpl(jobId, null, conf3, null, null, null, null, null,
+        null, null, null, true, null, 0, null);
+    Assert.assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB));
+    Assert.assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB));
+
+    // Setup configuration access without security enabled
+    Configuration conf4 = new Configuration();
+    conf4.setBoolean(MRConfig.MR_ACLS_ENABLED, false);
+    conf4.set(MRJobConfig.JOB_ACL_VIEW_JOB, "");
+
+    // Verify access
+    JobImpl job4 = new JobImpl(jobId, null, conf4, null, null, null, null, null,
+        null, null, null, true, null, 0, null);
+    Assert.assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB));
+    Assert.assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB));
+  }
 }

+ 0 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java

@@ -328,9 +328,6 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
   @Override
   public
       boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) {
-    if (!UserGroupInformation.isSecurityEnabled()) {
-      return true;
-    }
     Map<JobACL, AccessControlList> jobACLs = jobInfo.getJobACLs();
     AccessControlList jobACL = jobACLs.get(jobOperation);
     return aclsMgr.checkAccess(callerUGI, jobOperation, 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java

@@ -152,7 +152,7 @@ public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job {
 
   @Override
   public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) {
-    return false;
+    return true;
   }
   
   @Override