|
@@ -26,6 +26,7 @@ import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.io.DataInputByteBuffer;
|
|
|
+import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -33,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
+import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
@@ -55,6 +57,7 @@ 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.scheduler.SchedulerUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
|
|
@@ -78,7 +81,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|
|
private final YarnScheduler scheduler;
|
|
|
private final ApplicationACLsManager applicationACLsManager;
|
|
|
private Configuration conf;
|
|
|
-
|
|
|
+ private boolean isAclEnabled = false;
|
|
|
public RMAppManager(RMContext context,
|
|
|
YarnScheduler scheduler, ApplicationMasterService masterService,
|
|
|
ApplicationACLsManager applicationACLsManager, Configuration conf) {
|
|
@@ -97,6 +100,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|
|
if (this.maxCompletedAppsInStateStore > this.maxCompletedAppsInMemory) {
|
|
|
this.maxCompletedAppsInStateStore = this.maxCompletedAppsInMemory;
|
|
|
}
|
|
|
+ this.isAclEnabled = conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
|
|
|
+ YarnConfiguration.DEFAULT_YARN_ACL_ENABLE);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -276,7 +281,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|
|
@SuppressWarnings("unchecked")
|
|
|
protected void submitApplication(
|
|
|
ApplicationSubmissionContext submissionContext, long submitTime,
|
|
|
- String user) throws YarnException {
|
|
|
+ String user) throws YarnException, AccessControlException {
|
|
|
ApplicationId applicationId = submissionContext.getApplicationId();
|
|
|
|
|
|
RMAppImpl application =
|
|
@@ -325,7 +330,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|
|
|
|
|
private RMAppImpl createAndPopulateNewRMApp(
|
|
|
ApplicationSubmissionContext submissionContext, long submitTime,
|
|
|
- String user, boolean isRecovery) throws YarnException {
|
|
|
+ String user, boolean isRecovery)
|
|
|
+ throws YarnException, AccessControlException {
|
|
|
// Do queue mapping
|
|
|
if (!isRecovery) {
|
|
|
if (rmContext.getQueuePlacementManager() != null) {
|
|
@@ -346,6 +352,22 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|
|
submissionContext.getQueue(), applicationId);
|
|
|
submissionContext.setPriority(appPriority);
|
|
|
|
|
|
+ UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(user);
|
|
|
+ // Since FairScheduler queue mapping is done inside scheduler,
|
|
|
+ // if FairScheduler is used and the queue doesn't exist, we should not
|
|
|
+ // fail here because queue will be created inside FS. Ideally, FS queue
|
|
|
+ // mapping should be done outside scheduler too like CS.
|
|
|
+ // For now, exclude FS for the acl check.
|
|
|
+ if (!isRecovery && isAclEnabled && scheduler instanceof CapacityScheduler &&
|
|
|
+ !scheduler.checkAccess(userUgi, QueueACL.SUBMIT_APPLICATIONS,
|
|
|
+ submissionContext.getQueue()) &&
|
|
|
+ !scheduler.checkAccess(userUgi, QueueACL.ADMINISTER_QUEUE,
|
|
|
+ submissionContext.getQueue())) {
|
|
|
+ throw new AccessControlException(
|
|
|
+ "User " + user + " does not have permission to submit "
|
|
|
+ + applicationId + " to queue " + submissionContext.getQueue());
|
|
|
+ }
|
|
|
+
|
|
|
// Create RMApp
|
|
|
RMAppImpl application = new RMAppImpl(applicationId, rmContext, this.conf,
|
|
|
submissionContext.getApplicationName(), user,
|