YARN-4571. Make app id/name available to the yarn authorizer provider for better auditing. (Jian He via wangda)

This commit is contained in:
Wangda Tan 2016-01-13 13:18:31 +08:00
parent fbb5868deb
commit c0537bcd2c
15 changed files with 248 additions and 68 deletions

View File

@ -88,6 +88,9 @@ Release 2.9.0 - UNRELEASED
YARN-4438. Implement RM leader election with curator. (Jian He via xgong) YARN-4438. Implement RM leader election with curator. (Jian He via xgong)
YARN-4571. Make app id/name available to the yarn authorizer provider for
better auditing. (Jian He via wangda)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -278,7 +278,11 @@ public class YarnConfiguration extends Configuration {
public static final String YARN_ACL_ENABLE = public static final String YARN_ACL_ENABLE =
YARN_PREFIX + "acl.enable"; YARN_PREFIX + "acl.enable";
public static final boolean DEFAULT_YARN_ACL_ENABLE = false; public static final boolean DEFAULT_YARN_ACL_ENABLE = false;
public static boolean isAclEnabled(Configuration conf) {
return conf.getBoolean(YARN_ACL_ENABLE, DEFAULT_YARN_ACL_ENABLE);
}
/** ACL of who can be admin of YARN cluster.*/ /** ACL of who can be admin of YARN cluster.*/
public static final String YARN_ADMIN_ACL = public static final String YARN_ADMIN_ACL =
YARN_PREFIX + "admin.acl"; YARN_PREFIX + "admin.acl";

View File

@ -0,0 +1,70 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.security;
import org.apache.hadoop.security.UserGroupInformation;
/**
* This request object contains all the context information to determine whether
* a user has permission to access the target entity.
* user : the user who's currently accessing
* accessType : the access type against the entity.
* entity : the target object user is accessing.
* appId : the associated app Id for current access. This could be null
* if no app is associated.
* appName : the associated app name for current access. This could be null if
* no app is associated.
*/
public class AccessRequest {
private PrivilegedEntity entity;
private UserGroupInformation user;
private AccessType accessType;
private String appId;
private String appName;
public AccessRequest(PrivilegedEntity entity, UserGroupInformation user,
AccessType accessType, String appId, String appName) {
this.entity = entity;
this.user = user;
this.accessType = accessType;
this.appId = appId;
this.appName = appName;
}
public UserGroupInformation getUser() {
return user;
}
public AccessType getAccessType() {
return accessType;
}
public String getAppId() {
return appId;
}
public String getAppName() {
return appName;
}
public PrivilegedEntity getEntity() {
return entity;
}
}

View File

@ -18,9 +18,11 @@
package org.apache.hadoop.yarn.security; package org.apache.hadoop.yarn.security;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Unstable;
@ -38,10 +40,12 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
@Unstable @Unstable
public class ConfiguredYarnAuthorizer extends YarnAuthorizationProvider { public class ConfiguredYarnAuthorizer extends YarnAuthorizationProvider {
private final ConcurrentMap<PrivilegedEntity, Map<AccessType, AccessControlList>> allAcls = private final ConcurrentMap<PrivilegedEntity, Map<AccessType, AccessControlList>>
new ConcurrentHashMap<>(); allAcls = new ConcurrentHashMap<>();
private volatile AccessControlList adminAcl = null; private volatile AccessControlList adminAcl = null;
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();;
private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock();
private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock();
@Override @Override
public void init(Configuration conf) { public void init(Configuration conf) {
@ -51,13 +55,19 @@ public class ConfiguredYarnAuthorizer extends YarnAuthorizationProvider {
} }
@Override @Override
public void setPermission(PrivilegedEntity target, public void setPermission(List<Permission> permissions,
Map<AccessType, AccessControlList> acls, UserGroupInformation ugi) { UserGroupInformation user) {
allAcls.put(target, acls); try {
writeLock.lock();
for (Permission perm : permissions) {
allAcls.put(perm.getTarget(), perm.getAcls());
}
} finally {
writeLock.unlock();
}
} }
@Override private boolean checkPermissionInternal(AccessType accessType,
public boolean checkPermission(AccessType accessType,
PrivilegedEntity target, UserGroupInformation user) { PrivilegedEntity target, UserGroupInformation user) {
boolean ret = false; boolean ret = false;
Map<AccessType, AccessControlList> acls = allAcls.get(target); Map<AccessType, AccessControlList> acls = allAcls.get(target);
@ -74,13 +84,25 @@ public class ConfiguredYarnAuthorizer extends YarnAuthorizationProvider {
if (!queueName.contains(".")) { if (!queueName.contains(".")) {
return ret; return ret;
} }
String parentQueueName = queueName.substring(0, queueName.lastIndexOf(".")); String parentQueueName =
return checkPermission(accessType, new PrivilegedEntity(target.getType(), queueName.substring(0, queueName.lastIndexOf("."));
parentQueueName), user); return checkPermissionInternal(accessType,
new PrivilegedEntity(target.getType(), parentQueueName), user);
} }
return ret; return ret;
} }
@Override
public boolean checkPermission(AccessRequest accessRequest) {
try {
readLock.lock();
return checkPermissionInternal(accessRequest.getAccessType(),
accessRequest.getEntity(), accessRequest.getUser());
} finally {
readLock.unlock();
}
}
@Override @Override
public void setAdmins(AccessControlList acls, UserGroupInformation ugi) { public void setAdmins(AccessControlList acls, UserGroupInformation ugi) {
adminAcl = acls; adminAcl = acls;

View File

@ -0,0 +1,47 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.security;
import org.apache.hadoop.security.authorize.AccessControlList;
import java.util.Map;
/**
* This class contains permissions info for the target object.
*/
public class Permission {
private PrivilegedEntity target;
private Map<AccessType, AccessControlList> acls;
public Permission(PrivilegedEntity target,
Map<AccessType, AccessControlList> acls) {
this.target = target;
this.acls = acls;
}
public Map<AccessType, AccessControlList> getAcls() {
return acls;
}
public PrivilegedEntity getTarget() {
return target;
}
}

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.yarn.security; package org.apache.hadoop.yarn.security;
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.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -30,6 +28,8 @@ import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import java.util.List;
/** /**
* An implementation of the interface will provide authorization related * An implementation of the interface will provide authorization related
* information and enforce permission check. It is excepted that any of the * information and enforce permission check. It is excepted that any of the
@ -69,30 +69,22 @@ public abstract class YarnAuthorizationProvider {
/** /**
* Check if user has the permission to access the target object. * Check if user has the permission to access the target object.
* *
* @param accessType * @param accessRequest
* The type of accessing method. * the request object which contains all the access context info.
* @param target
* The target object being accessed, e.g. app/queue
* @param user
* User who access the target
* @return true if user can access the object, otherwise false. * @return true if user can access the object, otherwise false.
*/ */
public abstract boolean checkPermission(AccessType accessType,
PrivilegedEntity target, UserGroupInformation user); public abstract boolean checkPermission(AccessRequest accessRequest);
/** /**
* Set ACLs for the target object. AccessControlList class encapsulate the * Set permissions for the target object.
* users and groups who can access the target.
* *
* @param target * @param permissions
* The target object. * A list of permissions on the target object.
* @param acls
* A map from access method to a list of users and/or groups who has
* permission to do the access.
* @param ugi User who sets the permissions. * @param ugi User who sets the permissions.
*/ */
public abstract void setPermission(PrivilegedEntity target, public abstract void setPermission(List<Permission> permissions,
Map<AccessType, AccessControlList> acls, UserGroupInformation ugi); UserGroupInformation ugi);
/** /**
* Set a list of users/groups who have admin access * Set a list of users/groups who have admin access

View File

@ -292,12 +292,13 @@ public class ClientRMService extends AbstractService implements
* @return * @return
*/ */
private boolean checkAccess(UserGroupInformation callerUGI, String owner, private boolean checkAccess(UserGroupInformation callerUGI, String owner,
ApplicationAccessType operationPerformed, ApplicationAccessType operationPerformed, RMApp application) {
RMApp application) { return applicationsACLsManager
return applicationsACLsManager.checkAccess(callerUGI, operationPerformed, .checkAccess(callerUGI, operationPerformed, owner,
owner, application.getApplicationId()) application.getApplicationId()) || queueACLsManager
|| queueACLsManager.checkAccess(callerUGI, QueueACL.ADMINISTER_QUEUE, .checkAccess(callerUGI, QueueACL.ADMINISTER_QUEUE,
application.getQueue()); application.getQueue(), application.getApplicationId(),
application.getName());
} }
ApplicationId getNewApplicationId() { ApplicationId getNewApplicationId() {
@ -1386,7 +1387,7 @@ public class ClientRMService extends AbstractService implements
} }
// Check if user has access on the managed queue // Check if user has access on the managed queue
if (!queueACLsManager.checkAccess(callerUGI, QueueACL.SUBMIT_APPLICATIONS, if (!queueACLsManager.checkAccess(callerUGI, QueueACL.SUBMIT_APPLICATIONS,
queueName)) { queueName, null, null)) {
RMAuditLogger.logFailure( RMAuditLogger.logFailure(
callerUGI.getShortUserName(), callerUGI.getShortUserName(),
auditConstant, auditConstant,

View File

@ -17,11 +17,7 @@
*/ */
package org.apache.hadoop.yarn.server.resourcemanager; package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.IOException; import com.google.common.annotations.VisibleForTesting;
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;
@ -41,6 +37,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException; import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.ipc.RPCUtil; 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.RMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; 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.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; 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. * 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 YarnScheduler scheduler;
private final ApplicationACLsManager applicationACLsManager; private final ApplicationACLsManager applicationACLsManager;
private Configuration conf; private Configuration conf;
private boolean isAclEnabled = false; private YarnAuthorizationProvider authorizer;
public RMAppManager(RMContext context, public RMAppManager(RMContext context,
YarnScheduler scheduler, ApplicationMasterService masterService, YarnScheduler scheduler, ApplicationMasterService masterService,
ApplicationACLsManager applicationACLsManager, Configuration conf) { ApplicationACLsManager applicationACLsManager, Configuration conf) {
@ -100,8 +102,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
if (this.maxCompletedAppsInStateStore > this.maxCompletedAppsInMemory) { if (this.maxCompletedAppsInStateStore > this.maxCompletedAppsInMemory) {
this.maxCompletedAppsInStateStore = this.maxCompletedAppsInMemory; this.maxCompletedAppsInStateStore = this.maxCompletedAppsInMemory;
} }
this.isAclEnabled = conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE, this.authorizer = YarnAuthorizationProvider.getInstance(conf);
YarnConfiguration.DEFAULT_YARN_ACL_ENABLE);
} }
/** /**
@ -358,11 +359,20 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
// fail here because queue will be created inside FS. Ideally, FS queue // fail here because queue will be created inside FS. Ideally, FS queue
// 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 && isAclEnabled && scheduler instanceof CapacityScheduler && if (!isRecovery && YarnConfiguration.isAclEnabled(conf)
!scheduler.checkAccess(userUgi, QueueACL.SUBMIT_APPLICATIONS, && scheduler instanceof CapacityScheduler &&
submissionContext.getQueue()) && !authorizer.checkPermission(new AccessRequest(
!scheduler.checkAccess(userUgi, QueueACL.ADMINISTER_QUEUE, ((CapacityScheduler) scheduler)
submissionContext.getQueue())) { .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( throw new AccessControlException(
"User " + user + " does not have permission to submit " "User " + user + " does not have permission to submit "
+ applicationId + " to queue " + submissionContext.getQueue()); + applicationId + " to queue " + submissionContext.getQueue());

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.AccessRequest;
import org.apache.hadoop.yarn.security.AccessType; import org.apache.hadoop.yarn.security.AccessType;
import org.apache.hadoop.yarn.security.PrivilegedEntity; import org.apache.hadoop.yarn.security.PrivilegedEntity;
import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType; import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
@ -172,6 +173,7 @@ public abstract class AbstractCSQueue implements CSQueue {
return queueName; return queueName;
} }
@Override
public PrivilegedEntity getPrivilegedEntity() { public PrivilegedEntity getPrivilegedEntity() {
return queueEntity; return queueEntity;
} }
@ -192,8 +194,9 @@ public abstract class AbstractCSQueue implements CSQueue {
@Override @Override
public boolean hasAccess(QueueACL acl, UserGroupInformation user) { public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
return authorizer.checkPermission(SchedulerUtils.toAccessType(acl), return authorizer.checkPermission(
queueEntity, user); new AccessRequest(queueEntity, user, SchedulerUtils.toAccessType(acl),
null, null));
} }
@Override @Override

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.security.PrivilegedEntity;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
@ -73,7 +74,9 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
* @return the full name of the queue * @return the full name of the queue
*/ */
public String getQueuePath(); public String getQueuePath();
public PrivilegedEntity getPrivilegedEntity();
/** /**
* Get the configured <em>capacity</em> of the queue. * Get the configured <em>capacity</em> of the queue.
* @return configured queue capacity * @return configured queue capacity

View File

@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes; import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
import org.apache.hadoop.yarn.security.Permission;
import org.apache.hadoop.yarn.security.YarnAuthorizationProvider; import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
@ -533,11 +534,13 @@ public class CapacityScheduler extends
@VisibleForTesting @VisibleForTesting
public static void setQueueAcls(YarnAuthorizationProvider authorizer, public static void setQueueAcls(YarnAuthorizationProvider authorizer,
Map<String, CSQueue> queues) throws IOException { Map<String, CSQueue> queues) throws IOException {
List<Permission> permissions = new ArrayList<>();
for (CSQueue queue : queues.values()) { for (CSQueue queue : queues.values()) {
AbstractCSQueue csQueue = (AbstractCSQueue) queue; AbstractCSQueue csQueue = (AbstractCSQueue) queue;
authorizer.setPermission(csQueue.getPrivilegedEntity(), permissions.add(
csQueue.getACLs(), UserGroupInformation.getCurrentUser()); new Permission(csQueue.getPrivilegedEntity(), csQueue.getACLs()));
} }
authorizer.setPermission(permissions, UserGroupInformation.getCurrentUser());
} }
private Map<String, Set<String>> getQueueToLabels() { private Map<String, Set<String>> getQueueToLabels() {

View File

@ -20,16 +20,22 @@ package org.apache.hadoop.yarn.server.resourcemanager.security;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.AccessRequest;
import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
public class QueueACLsManager { public class QueueACLsManager {
private ResourceScheduler scheduler; private ResourceScheduler scheduler;
private boolean isACLsEnable; private boolean isACLsEnable;
private YarnAuthorizationProvider authorizer;
@VisibleForTesting @VisibleForTesting
public QueueACLsManager() { public QueueACLsManager() {
this(null, new Configuration()); this(null, new Configuration());
@ -39,13 +45,21 @@ public class QueueACLsManager {
this.scheduler = scheduler; this.scheduler = scheduler;
this.isACLsEnable = conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE, this.isACLsEnable = conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
YarnConfiguration.DEFAULT_YARN_ACL_ENABLE); YarnConfiguration.DEFAULT_YARN_ACL_ENABLE);
this.authorizer = YarnAuthorizationProvider.getInstance(conf);
} }
public boolean checkAccess(UserGroupInformation callerUGI, public boolean checkAccess(UserGroupInformation callerUGI, QueueACL acl,
QueueACL acl, String queueName) { String queueName, ApplicationId appId, String appName) {
if (!isACLsEnable) { if (!isACLsEnable) {
return true; return true;
} }
return scheduler.checkAccess(callerUGI, acl, queueName); if (scheduler instanceof CapacityScheduler) {
return authorizer.checkPermission(new AccessRequest(
((CapacityScheduler) scheduler).getQueue(queueName)
.getPrivilegedEntity(), callerUGI,
SchedulerUtils.toAccessType(acl), appId.toString(), appName));
} else {
return scheduler.checkAccess(callerUGI, acl, queueName);
}
} }
} }

View File

@ -225,7 +225,8 @@ public class RMWebServices extends WebServices {
ApplicationAccessType.VIEW_APP, app.getUser(), ApplicationAccessType.VIEW_APP, app.getUser(),
app.getApplicationId()) || app.getApplicationId()) ||
this.rm.getQueueACLsManager().checkAccess(callerUGI, this.rm.getQueueACLsManager().checkAccess(callerUGI,
QueueACL.ADMINISTER_QUEUE, app.getQueue()))) { QueueACL.ADMINISTER_QUEUE, app.getQueue(),
app.getApplicationId(), app.getName()))) {
return false; return false;
} }
return true; return true;

View File

@ -102,6 +102,7 @@ public class TestApplicationACLs {
AccessControlList adminACL = new AccessControlList(""); AccessControlList adminACL = new AccessControlList("");
adminACL.addGroup(SUPER_GROUP); adminACL.addGroup(SUPER_GROUP);
conf.set(YarnConfiguration.YARN_ADMIN_ACL, adminACL.getAclString()); conf.set(YarnConfiguration.YARN_ADMIN_ACL, adminACL.getAclString());
resourceManager = new MockRM(conf) { resourceManager = new MockRM(conf) {
@Override @Override
@ -110,7 +111,8 @@ public class TestApplicationACLs {
Configuration conf) { Configuration conf) {
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class); QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class), when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class),
any(QueueACL.class), anyString())).thenAnswer(new Answer() { any(QueueACL.class), anyString(), any(ApplicationId.class),
anyString())).thenAnswer(new Answer() {
public Object answer(InvocationOnMock invocation) { public Object answer(InvocationOnMock invocation) {
return isQueueUser; return isQueueUser;
} }

View File

@ -467,7 +467,8 @@ public class TestClientRMService {
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class); QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
when( when(
mockQueueACLsManager.checkAccess(any(UserGroupInformation.class), mockQueueACLsManager.checkAccess(any(UserGroupInformation.class),
any(QueueACL.class), anyString())).thenReturn(true); any(QueueACL.class), anyString(), any(ApplicationId.class),
anyString())).thenReturn(true);
return new ClientRMService(rmContext, yarnScheduler, appManager, return new ClientRMService(rmContext, yarnScheduler, appManager,
mockAclsManager, mockQueueACLsManager, null); mockAclsManager, mockQueueACLsManager, null);
} }
@ -568,7 +569,8 @@ public class TestClientRMService {
ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class); ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class);
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class); QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class), when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class),
any(QueueACL.class), anyString())).thenReturn(true); any(QueueACL.class), anyString(), any(ApplicationId.class),
anyString())).thenReturn(true);
when(mockAclsManager.checkAccess(any(UserGroupInformation.class), when(mockAclsManager.checkAccess(any(UserGroupInformation.class),
any(ApplicationAccessType.class), anyString(), any(ApplicationAccessType.class), anyString(),
any(ApplicationId.class))).thenReturn(true); any(ApplicationId.class))).thenReturn(true);
@ -594,7 +596,8 @@ public class TestClientRMService {
QueueACLsManager mockQueueACLsManager1 = QueueACLsManager mockQueueACLsManager1 =
mock(QueueACLsManager.class); mock(QueueACLsManager.class);
when(mockQueueACLsManager1.checkAccess(any(UserGroupInformation.class), when(mockQueueACLsManager1.checkAccess(any(UserGroupInformation.class),
any(QueueACL.class), anyString())).thenReturn(false); any(QueueACL.class), anyString(), any(ApplicationId.class),
anyString())).thenReturn(false);
when(mockAclsManager1.checkAccess(any(UserGroupInformation.class), when(mockAclsManager1.checkAccess(any(UserGroupInformation.class),
any(ApplicationAccessType.class), anyString(), any(ApplicationAccessType.class), anyString(),
any(ApplicationId.class))).thenReturn(false); any(ApplicationId.class))).thenReturn(false);
@ -633,7 +636,8 @@ public class TestClientRMService {
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class); QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class), when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class),
any(QueueACL.class), anyString())).thenReturn(true); any(QueueACL.class), anyString(), any(ApplicationId.class),
anyString())).thenReturn(true);
ClientRMService rmService = ClientRMService rmService =
new ClientRMService(rmContext, yarnScheduler, appManager, new ClientRMService(rmContext, yarnScheduler, appManager,
mockAclsManager, mockQueueACLsManager, null); mockAclsManager, mockQueueACLsManager, null);
@ -721,7 +725,8 @@ public class TestClientRMService {
ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class); ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class);
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class); QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class), when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class),
any(QueueACL.class), anyString())).thenReturn(true); any(QueueACL.class), anyString(), any(ApplicationId.class),
anyString())).thenReturn(true);
ClientRMService rmService = ClientRMService rmService =
new ClientRMService(rmContext, yarnScheduler, appManager, new ClientRMService(rmContext, yarnScheduler, appManager,
mockAclsManager, mockQueueACLsManager, null); mockAclsManager, mockQueueACLsManager, null);