|
@@ -17,7 +17,11 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.yarn.server.resourcemanager;
|
|
package org.apache.hadoop.yarn.server.resourcemanager;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
|
+import java.nio.ByteBuffer;
|
|
|
|
+import java.util.LinkedList;
|
|
|
|
+import java.util.Map;
|
|
|
|
+
|
|
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;
|
|
@@ -55,14 +59,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
|
-import java.nio.ByteBuffer;
|
|
|
|
-import java.util.LinkedList;
|
|
|
|
-import java.util.Map;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
/**
|
|
/**
|
|
* This class manages the list of applications for the resource manager.
|
|
* This class manages the list of applications for the resource manager.
|
|
@@ -360,22 +362,23 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|
// mapping should be done outside scheduler too like CS.
|
|
// mapping should be done outside scheduler too like CS.
|
|
// For now, exclude FS for the acl check.
|
|
// For now, exclude FS for the acl check.
|
|
if (!isRecovery && YarnConfiguration.isAclEnabled(conf)
|
|
if (!isRecovery && YarnConfiguration.isAclEnabled(conf)
|
|
- && scheduler instanceof CapacityScheduler &&
|
|
|
|
- !authorizer.checkPermission(new AccessRequest(
|
|
|
|
- ((CapacityScheduler) scheduler)
|
|
|
|
- .getQueue(submissionContext.getQueue()).getPrivilegedEntity(),
|
|
|
|
- userUgi, SchedulerUtils.toAccessType(QueueACL.SUBMIT_APPLICATIONS),
|
|
|
|
- submissionContext.getApplicationId().toString(),
|
|
|
|
- submissionContext.getApplicationName())) &&
|
|
|
|
- !authorizer.checkPermission(new AccessRequest(
|
|
|
|
- ((CapacityScheduler) scheduler)
|
|
|
|
- .getQueue(submissionContext.getQueue()).getPrivilegedEntity(),
|
|
|
|
- userUgi, SchedulerUtils.toAccessType(QueueACL.ADMINISTER_QUEUE),
|
|
|
|
- submissionContext.getApplicationId().toString(),
|
|
|
|
- submissionContext.getApplicationName()))) {
|
|
|
|
- throw new AccessControlException(
|
|
|
|
- "User " + user + " does not have permission to submit "
|
|
|
|
- + applicationId + " to queue " + submissionContext.getQueue());
|
|
|
|
|
|
+ && scheduler instanceof CapacityScheduler) {
|
|
|
|
+ String queueName = submissionContext.getQueue();
|
|
|
|
+ String appName = submissionContext.getApplicationName();
|
|
|
|
+ CSQueue csqueue = ((CapacityScheduler) scheduler).getQueue(queueName);
|
|
|
|
+ if (null != csqueue
|
|
|
|
+ && !authorizer.checkPermission(
|
|
|
|
+ new AccessRequest(csqueue.getPrivilegedEntity(), userUgi,
|
|
|
|
+ SchedulerUtils.toAccessType(QueueACL.SUBMIT_APPLICATIONS),
|
|
|
|
+ applicationId.toString(), appName))
|
|
|
|
+ && !authorizer.checkPermission(
|
|
|
|
+ new AccessRequest(csqueue.getPrivilegedEntity(), userUgi,
|
|
|
|
+ SchedulerUtils.toAccessType(QueueACL.ADMINISTER_QUEUE),
|
|
|
|
+ applicationId.toString(), appName))) {
|
|
|
|
+ throw new AccessControlException(
|
|
|
|
+ "User " + user + " does not have permission to submit "
|
|
|
|
+ + applicationId + " to queue " + submissionContext.getQueue());
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
// Create RMApp
|
|
// Create RMApp
|