YARN-4571. Make app id/name available to the yarn authorizer provider for better auditing. (Jian He via wangda)
This commit is contained in:
parent
fbb5868deb
commit
c0537bcd2c
|
@ -88,6 +88,9 @@ Release 2.9.0 - UNRELEASED
|
|||
|
||||
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
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -278,7 +278,11 @@ public class YarnConfiguration extends Configuration {
|
|||
public static final String YARN_ACL_ENABLE =
|
||||
YARN_PREFIX + "acl.enable";
|
||||
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.*/
|
||||
public static final String YARN_ADMIN_ACL =
|
||||
YARN_PREFIX + "admin.acl";
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -18,9 +18,11 @@
|
|||
|
||||
package org.apache.hadoop.yarn.security;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
|
@ -38,10 +40,12 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
|
|||
@Unstable
|
||||
public class ConfiguredYarnAuthorizer extends YarnAuthorizationProvider {
|
||||
|
||||
private final ConcurrentMap<PrivilegedEntity, Map<AccessType, AccessControlList>> allAcls =
|
||||
new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<PrivilegedEntity, Map<AccessType, AccessControlList>>
|
||||
allAcls = new ConcurrentHashMap<>();
|
||||
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
|
||||
public void init(Configuration conf) {
|
||||
|
@ -51,13 +55,19 @@ public class ConfiguredYarnAuthorizer extends YarnAuthorizationProvider {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setPermission(PrivilegedEntity target,
|
||||
Map<AccessType, AccessControlList> acls, UserGroupInformation ugi) {
|
||||
allAcls.put(target, acls);
|
||||
public void setPermission(List<Permission> permissions,
|
||||
UserGroupInformation user) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
for (Permission perm : permissions) {
|
||||
allAcls.put(perm.getTarget(), perm.getAcls());
|
||||
}
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean checkPermission(AccessType accessType,
|
||||
private boolean checkPermissionInternal(AccessType accessType,
|
||||
PrivilegedEntity target, UserGroupInformation user) {
|
||||
boolean ret = false;
|
||||
Map<AccessType, AccessControlList> acls = allAcls.get(target);
|
||||
|
@ -74,13 +84,25 @@ public class ConfiguredYarnAuthorizer extends YarnAuthorizationProvider {
|
|||
if (!queueName.contains(".")) {
|
||||
return ret;
|
||||
}
|
||||
String parentQueueName = queueName.substring(0, queueName.lastIndexOf("."));
|
||||
return checkPermission(accessType, new PrivilegedEntity(target.getType(),
|
||||
parentQueueName), user);
|
||||
String parentQueueName =
|
||||
queueName.substring(0, queueName.lastIndexOf("."));
|
||||
return checkPermissionInternal(accessType,
|
||||
new PrivilegedEntity(target.getType(), parentQueueName), user);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean checkPermission(AccessRequest accessRequest) {
|
||||
try {
|
||||
readLock.lock();
|
||||
return checkPermissionInternal(accessRequest.getAccessType(),
|
||||
accessRequest.getEntity(), accessRequest.getUser());
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAdmins(AccessControlList acls, UserGroupInformation ugi) {
|
||||
adminAcl = acls;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.yarn.security;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.yarn.conf.YarnConfiguration;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* An implementation of the interface will provide authorization related
|
||||
* 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.
|
||||
*
|
||||
* @param accessType
|
||||
* The type of accessing method.
|
||||
* @param target
|
||||
* The target object being accessed, e.g. app/queue
|
||||
* @param user
|
||||
* User who access the target
|
||||
* @param accessRequest
|
||||
* the request object which contains all the access context info.
|
||||
* @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
|
||||
* users and groups who can access the target.
|
||||
* Set permissions for the target object.
|
||||
*
|
||||
* @param target
|
||||
* 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 permissions
|
||||
* A list of permissions on the target object.
|
||||
* @param ugi User who sets the permissions.
|
||||
*/
|
||||
public abstract void setPermission(PrivilegedEntity target,
|
||||
Map<AccessType, AccessControlList> acls, UserGroupInformation ugi);
|
||||
public abstract void setPermission(List<Permission> permissions,
|
||||
UserGroupInformation ugi);
|
||||
|
||||
/**
|
||||
* Set a list of users/groups who have admin access
|
||||
|
|
|
@ -292,12 +292,13 @@ public class ClientRMService extends AbstractService implements
|
|||
* @return
|
||||
*/
|
||||
private boolean checkAccess(UserGroupInformation callerUGI, String owner,
|
||||
ApplicationAccessType operationPerformed,
|
||||
RMApp application) {
|
||||
return applicationsACLsManager.checkAccess(callerUGI, operationPerformed,
|
||||
owner, application.getApplicationId())
|
||||
|| queueACLsManager.checkAccess(callerUGI, QueueACL.ADMINISTER_QUEUE,
|
||||
application.getQueue());
|
||||
ApplicationAccessType operationPerformed, RMApp application) {
|
||||
return applicationsACLsManager
|
||||
.checkAccess(callerUGI, operationPerformed, owner,
|
||||
application.getApplicationId()) || queueACLsManager
|
||||
.checkAccess(callerUGI, QueueACL.ADMINISTER_QUEUE,
|
||||
application.getQueue(), application.getApplicationId(),
|
||||
application.getName());
|
||||
}
|
||||
|
||||
ApplicationId getNewApplicationId() {
|
||||
|
@ -1386,7 +1387,7 @@ public class ClientRMService extends AbstractService implements
|
|||
}
|
||||
// Check if user has access on the managed queue
|
||||
if (!queueACLsManager.checkAccess(callerUGI, QueueACL.SUBMIT_APPLICATIONS,
|
||||
queueName)) {
|
||||
queueName, null, null)) {
|
||||
RMAuditLogger.logFailure(
|
||||
callerUGI.getShortUserName(),
|
||||
auditConstant,
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
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.PrivilegedEntity;
|
||||
import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
|
||||
|
@ -172,6 +173,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|||
return queueName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PrivilegedEntity getPrivilegedEntity() {
|
||||
return queueEntity;
|
||||
}
|
||||
|
@ -192,8 +194,9 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|||
|
||||
@Override
|
||||
public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
|
||||
return authorizer.checkPermission(SchedulerUtils.toAccessType(acl),
|
||||
queueEntity, user);
|
||||
return authorizer.checkPermission(
|
||||
new AccessRequest(queueEntity, user, SchedulerUtils.toAccessType(acl),
|
||||
null, null));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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.QueueState;
|
||||
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.RMContainerEventType;
|
||||
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
|
||||
*/
|
||||
public String getQueuePath();
|
||||
|
||||
|
||||
public PrivilegedEntity getPrivilegedEntity();
|
||||
|
||||
/**
|
||||
* Get the configured <em>capacity</em> of the queue.
|
||||
* @return configured queue capacity
|
||||
|
|
|
@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
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.server.resourcemanager.RMContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||
|
@ -533,11 +534,13 @@ public class CapacityScheduler extends
|
|||
@VisibleForTesting
|
||||
public static void setQueueAcls(YarnAuthorizationProvider authorizer,
|
||||
Map<String, CSQueue> queues) throws IOException {
|
||||
List<Permission> permissions = new ArrayList<>();
|
||||
for (CSQueue queue : queues.values()) {
|
||||
AbstractCSQueue csQueue = (AbstractCSQueue) queue;
|
||||
authorizer.setPermission(csQueue.getPrivilegedEntity(),
|
||||
csQueue.getACLs(), UserGroupInformation.getCurrentUser());
|
||||
permissions.add(
|
||||
new Permission(csQueue.getPrivilegedEntity(), csQueue.getACLs()));
|
||||
}
|
||||
authorizer.setPermission(permissions, UserGroupInformation.getCurrentUser());
|
||||
}
|
||||
|
||||
private Map<String, Set<String>> getQueueToLabels() {
|
||||
|
|
|
@ -20,16 +20,22 @@ package org.apache.hadoop.yarn.server.resourcemanager.security;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.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 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 {
|
||||
private ResourceScheduler scheduler;
|
||||
private boolean isACLsEnable;
|
||||
|
||||
private YarnAuthorizationProvider authorizer;
|
||||
|
||||
@VisibleForTesting
|
||||
public QueueACLsManager() {
|
||||
this(null, new Configuration());
|
||||
|
@ -39,13 +45,21 @@ public class QueueACLsManager {
|
|||
this.scheduler = scheduler;
|
||||
this.isACLsEnable = conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
|
||||
YarnConfiguration.DEFAULT_YARN_ACL_ENABLE);
|
||||
this.authorizer = YarnAuthorizationProvider.getInstance(conf);
|
||||
}
|
||||
|
||||
public boolean checkAccess(UserGroupInformation callerUGI,
|
||||
QueueACL acl, String queueName) {
|
||||
public boolean checkAccess(UserGroupInformation callerUGI, QueueACL acl,
|
||||
String queueName, ApplicationId appId, String appName) {
|
||||
if (!isACLsEnable) {
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -225,7 +225,8 @@ public class RMWebServices extends WebServices {
|
|||
ApplicationAccessType.VIEW_APP, app.getUser(),
|
||||
app.getApplicationId()) ||
|
||||
this.rm.getQueueACLsManager().checkAccess(callerUGI,
|
||||
QueueACL.ADMINISTER_QUEUE, app.getQueue()))) {
|
||||
QueueACL.ADMINISTER_QUEUE, app.getQueue(),
|
||||
app.getApplicationId(), app.getName()))) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
|
|
|
@ -102,6 +102,7 @@ public class TestApplicationACLs {
|
|||
AccessControlList adminACL = new AccessControlList("");
|
||||
adminACL.addGroup(SUPER_GROUP);
|
||||
conf.set(YarnConfiguration.YARN_ADMIN_ACL, adminACL.getAclString());
|
||||
|
||||
resourceManager = new MockRM(conf) {
|
||||
|
||||
@Override
|
||||
|
@ -110,7 +111,8 @@ public class TestApplicationACLs {
|
|||
Configuration conf) {
|
||||
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.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) {
|
||||
return isQueueUser;
|
||||
}
|
||||
|
|
|
@ -467,7 +467,8 @@ public class TestClientRMService {
|
|||
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
|
||||
when(
|
||||
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,
|
||||
mockAclsManager, mockQueueACLsManager, null);
|
||||
}
|
||||
|
@ -568,7 +569,8 @@ public class TestClientRMService {
|
|||
ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class);
|
||||
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.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),
|
||||
any(ApplicationAccessType.class), anyString(),
|
||||
any(ApplicationId.class))).thenReturn(true);
|
||||
|
@ -594,7 +596,8 @@ public class TestClientRMService {
|
|||
QueueACLsManager mockQueueACLsManager1 =
|
||||
mock(QueueACLsManager.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),
|
||||
any(ApplicationAccessType.class), anyString(),
|
||||
any(ApplicationId.class))).thenReturn(false);
|
||||
|
@ -633,7 +636,8 @@ public class TestClientRMService {
|
|||
|
||||
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.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 =
|
||||
new ClientRMService(rmContext, yarnScheduler, appManager,
|
||||
mockAclsManager, mockQueueACLsManager, null);
|
||||
|
@ -721,7 +725,8 @@ public class TestClientRMService {
|
|||
ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class);
|
||||
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.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 =
|
||||
new ClientRMService(rmContext, yarnScheduler, appManager,
|
||||
mockAclsManager, mockQueueACLsManager, null);
|
||||
|
|
Loading…
Reference in New Issue