|
@@ -22,8 +22,6 @@ 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;
|
|
@@ -89,38 +87,30 @@ class ACLsManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Check the ACLs for a user doing the passed queue-operation and the passed
|
|
|
- * job operation.
|
|
|
+ * Check the ACLs for a user doing the passed 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) cluster
|
|
|
- * administrators (c) members of queue admins acl for the queue.</li>
|
|
|
+ * administrators (c) members of queue-submit-job-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) cluster administrators,
|
|
|
* (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
|
|
|
+ * @param job the job on which operation is requested
|
|
|
+ * @param callerUGI the user who is requesting the operation
|
|
|
+ * @param operation the operation requested
|
|
|
* @throws AccessControlException
|
|
|
- * @throws IOException
|
|
|
*/
|
|
|
- void checkAccess(JobInProgress job,
|
|
|
- UserGroupInformation callerUGI, QueueOperation qOperation,
|
|
|
- JobACL jobOperation, String operationName) throws AccessControlException {
|
|
|
+ void checkAccess(JobInProgress job, UserGroupInformation callerUGI,
|
|
|
+ Operation operation) 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, operationName);
|
|
|
+ checkAccess(jobStatus, callerUGI, queue, operation);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -133,132 +123,95 @@ class ACLsManager {
|
|
|
* </ul>
|
|
|
*/
|
|
|
void checkAccess(JobStatus jobStatus, UserGroupInformation callerUGI,
|
|
|
- JobACL jobOperation, String operationName) throws AccessControlException {
|
|
|
+ String queue, Operation operation)
|
|
|
+ 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,
|
|
|
- operationName);
|
|
|
- }
|
|
|
+ AccessControlList jobAcl =
|
|
|
+ jobStatus.getJobACLs().get(operation.jobACLNeeded);
|
|
|
|
|
|
- /**
|
|
|
- * 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) cluster administrators,
|
|
|
- * (d) members of job acl for the jobOperation</li>
|
|
|
- * </ul>
|
|
|
- */
|
|
|
- void checkAccess(String jobId, UserGroupInformation callerUGI,
|
|
|
- JobACL jobOperation, String jobOwner, AccessControlList jobAcl,
|
|
|
- String operationName)
|
|
|
- throws AccessControlException {
|
|
|
- // TODO: Queue admins are to be allowed to do the job view operation.
|
|
|
- checkAccess(jobId, callerUGI, null, null, jobOperation, jobOwner, jobAcl,
|
|
|
- operationName);
|
|
|
+ // If acls are enabled, check if callerUGI is jobOwner, queue admin,
|
|
|
+ // cluster admin or part of job ACL
|
|
|
+ checkAccess(jobId, callerUGI, queue, operation, jobOwner, jobAcl);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Check the ACLs for a user doing the passed queue-operation and the passed
|
|
|
- * job operation.
|
|
|
+ * Check the ACLs for a user doing the passed 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)cluster
|
|
|
- * administrators and (c) members of queue admins acl for the queue.</li>
|
|
|
+ * administrators and (c) members of queue-submit-job-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) cluster administrators,
|
|
|
* (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.
|
|
|
+ * callerUGI is the user who is trying to perform the operation.
|
|
|
+ * jobAcl could be job-view-acl or job-modify-acl depending on job operation.
|
|
|
*/
|
|
|
void checkAccess(String jobId, UserGroupInformation callerUGI,
|
|
|
- String queue, QueueOperation qOperation,
|
|
|
- JobACL jobOperation, String jobOwner, AccessControlList jobAcl,
|
|
|
- String operationName)
|
|
|
- throws AccessControlException {
|
|
|
+ String queue, Operation operation, String jobOwner,
|
|
|
+ AccessControlList jobAcl) throws AccessControlException {
|
|
|
if (!aclsEnabled) {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
String user = callerUGI.getShortUserName();
|
|
|
+ String targetResource = jobId + " in queue " + queue;
|
|
|
|
|
|
// Allow mapreduce cluster admins to do any queue operation and
|
|
|
// any job operation
|
|
|
if (isMRAdmin(callerUGI)) {
|
|
|
- if (qOperation == QueueOperation.SUBMIT_JOB) {
|
|
|
- AuditLogger.logSuccess(user, operationName, queue);
|
|
|
- } else {
|
|
|
- AuditLogger.logSuccess(user, operationName, jobId);
|
|
|
- }
|
|
|
+ AuditLogger.logSuccess(user, operation.name(), targetResource);
|
|
|
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, operationName, null, queue,
|
|
|
- Constants.UNAUTHORIZED_USER + ", job : " + jobId);
|
|
|
+ if (operation == Operation.SUBMIT_JOB) {
|
|
|
+ // This is strictly queue operation(not a job operation)
|
|
|
+ if (!queueManager.hasAccess(queue, operation.qACLNeeded, callerUGI)) {
|
|
|
+ AuditLogger.logFailure(user, operation.name(),
|
|
|
+ queueManager.getQueueACL(queue, operation.qACLNeeded).toString(),
|
|
|
+ targetResource, Constants.UNAUTHORIZED_USER);
|
|
|
|
|
|
throw new AccessControlException("User "
|
|
|
+ callerUGI.getShortUserName() + " cannot perform "
|
|
|
- + "operation " + operationName + " on queue " + queue
|
|
|
+ + "operation " + operation.name() + " on queue " + queue
|
|
|
+ ".\n Please run \"hadoop queue -showacls\" "
|
|
|
+ "command to find the queues you have access to .");
|
|
|
} else {
|
|
|
- AuditLogger.logSuccess(user, operationName, queue);
|
|
|
+ AuditLogger.logSuccess(user, operation.name(), targetResource);
|
|
|
return;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- if (jobOperation == JobACL.VIEW_JOB) {
|
|
|
- // check if jobOwner or part of acl-view-job
|
|
|
- if (jobACLsManager.checkAccess(callerUGI, jobOperation,
|
|
|
+ // Check if callerUGI is queueAdmin, jobOwner or part of job-acl.
|
|
|
+ // queueManager and queue are null only when called from
|
|
|
+ // TaskTracker(i.e. from TaskLogServlet) for the operation VIEW_TASK_LOGS.
|
|
|
+ // Caller of this method takes care of checking if callerUGI is a
|
|
|
+ // queue administrator for that operation.
|
|
|
+ if (operation == Operation.VIEW_TASK_LOGS) {
|
|
|
+ if (jobACLsManager.checkAccess(callerUGI, operation.jobACLNeeded,
|
|
|
jobOwner, jobAcl)) {
|
|
|
- AuditLogger.logSuccess(user, operationName, jobId.toString());
|
|
|
+ AuditLogger.logSuccess(user, operation.name(), targetResource);
|
|
|
return;
|
|
|
}
|
|
|
- else {
|
|
|
- AuditLogger.logFailure(user, operationName, null,
|
|
|
- jobId.toString(), Constants.UNAUTHORIZED_USER);
|
|
|
- throw new AccessControlException("User "
|
|
|
- + callerUGI.getShortUserName() + " cannot perform operation "
|
|
|
- + operationName + " on " + jobId);
|
|
|
- }
|
|
|
+ } else if (queueManager.hasAccess(queue, operation.qACLNeeded, callerUGI) ||
|
|
|
+ jobACLsManager.checkAccess(callerUGI, operation.jobACLNeeded,
|
|
|
+ jobOwner, jobAcl)) {
|
|
|
+ AuditLogger.logSuccess(user, operation.name(), targetResource);
|
|
|
+ return;
|
|
|
}
|
|
|
|
|
|
- if (jobOperation == JobACL.MODIFY_JOB) {
|
|
|
- // check if queueAdmin, jobOwner or part of acl-modify-job
|
|
|
- if (queueManager.hasAccess(queue, qOperation, callerUGI)) {
|
|
|
- AuditLogger.logSuccess(user, operationName, queue);
|
|
|
- return;
|
|
|
- } else if (jobACLsManager.checkAccess(callerUGI, jobOperation,
|
|
|
- jobOwner, jobAcl)) {
|
|
|
- AuditLogger.logSuccess(user, operationName, jobId);
|
|
|
- return;
|
|
|
- }
|
|
|
- AuditLogger.logFailure(user, operationName, null,
|
|
|
- jobId.toString(), Constants.UNAUTHORIZED_USER + ", queue : "
|
|
|
- + queue);
|
|
|
-
|
|
|
- throw new AccessControlException("User "
|
|
|
- + callerUGI.getShortUserName() + " cannot perform operation "
|
|
|
- + operationName + " on " + jobId + " that is in the queue "
|
|
|
- + queue);
|
|
|
- }
|
|
|
+ AuditLogger.logFailure(user, operation.name(), jobAcl.toString(),
|
|
|
+ targetResource, Constants.UNAUTHORIZED_USER);
|
|
|
|
|
|
- throw new AccessControlException("Unsupported queue operation "
|
|
|
- + qOperation + " on queue " + queue + ", job operation "
|
|
|
- + jobOperation + " on job " + jobId + " and the actual-operation "
|
|
|
- + operationName);
|
|
|
+ throw new AccessControlException("User "
|
|
|
+ + callerUGI.getShortUserName() + " cannot perform operation "
|
|
|
+ + operation.name() + " on " + jobId + " that is in the queue "
|
|
|
+ + queue);
|
|
|
}
|
|
|
|
|
|
}
|