|
@@ -17,11 +17,7 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
-import java.nio.ByteBuffer;
|
|
|
-import java.util.LinkedList;
|
|
|
-import java.util.Map;
|
|
|
-
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -41,6 +37,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
|
|
+import org.apache.hadoop.yarn.security.AccessRequest;
|
|
|
+import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
|
@@ -61,7 +59,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
|
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
+import java.io.IOException;
|
|
|
+import java.nio.ByteBuffer;
|
|
|
+import java.util.LinkedList;
|
|
|
+import java.util.Map;
|
|
|
|
|
|
/**
|
|
|
* This class manages the list of applications for the resource manager.
|
|
@@ -81,7 +82,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|
|
private final YarnScheduler scheduler;
|
|
|
private final ApplicationACLsManager applicationACLsManager;
|
|
|
private Configuration conf;
|
|
|
- private boolean isAclEnabled = false;
|
|
|
+ private YarnAuthorizationProvider authorizer;
|
|
|
+
|
|
|
public RMAppManager(RMContext context,
|
|
|
YarnScheduler scheduler, ApplicationMasterService masterService,
|
|
|
ApplicationACLsManager applicationACLsManager, Configuration conf) {
|
|
@@ -100,8 +102,7 @@ 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);
|
|
|
+ this.authorizer = YarnAuthorizationProvider.getInstance(conf);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -358,11 +359,20 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|
|
// 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())) {
|
|
|
+ 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());
|