|
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Matchers.anyBoolean;
|
|
|
+import static org.mockito.Matchers.anyListOf;
|
|
|
import static org.mockito.Matchers.anyString;
|
|
|
import static org.mockito.Matchers.eq;
|
|
|
import static org.mockito.Mockito.doReturn;
|
|
@@ -33,6 +34,8 @@ import java.io.File;
|
|
|
import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.security.AccessControlException;
|
|
|
+import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.EnumSet;
|
|
@@ -155,6 +158,8 @@ import org.junit.Test;
|
|
|
|
|
|
import com.google.common.collect.ImmutableSet;
|
|
|
import com.google.common.collect.Sets;
|
|
|
+import org.mockito.invocation.InvocationOnMock;
|
|
|
+import org.mockito.stubbing.Answer;
|
|
|
|
|
|
public class TestClientRMService {
|
|
|
|
|
@@ -572,10 +577,261 @@ public class TestClientRMService {
|
|
|
ApplicationId applicationId =
|
|
|
BuilderUtils.newApplicationId(System.currentTimeMillis(), 0);
|
|
|
MoveApplicationAcrossQueuesRequest request =
|
|
|
- MoveApplicationAcrossQueuesRequest.newInstance(applicationId, "newqueue");
|
|
|
+ MoveApplicationAcrossQueuesRequest.newInstance(applicationId,
|
|
|
+ "newqueue");
|
|
|
rmService.moveApplicationAcrossQueues(request);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testMoveApplicationSubmitTargetQueue() throws Exception {
|
|
|
+ // move the application as the owner
|
|
|
+ ApplicationId applicationId = getApplicationId(1);
|
|
|
+ UserGroupInformation aclUGI = UserGroupInformation.getCurrentUser();
|
|
|
+ QueueACLsManager queueACLsManager = getQueueAclManager("allowed_queue",
|
|
|
+ QueueACL.SUBMIT_APPLICATIONS, aclUGI);
|
|
|
+ ApplicationACLsManager appAclsManager = getAppAclManager();
|
|
|
+
|
|
|
+ ClientRMService rmService = createClientRMServiceForMoveApplicationRequest(
|
|
|
+ applicationId, aclUGI.getShortUserName(), appAclsManager,
|
|
|
+ queueACLsManager);
|
|
|
+
|
|
|
+ // move as the owner queue in the acl
|
|
|
+ MoveApplicationAcrossQueuesRequest moveAppRequest =
|
|
|
+ MoveApplicationAcrossQueuesRequest.
|
|
|
+ newInstance(applicationId, "allowed_queue");
|
|
|
+ rmService.moveApplicationAcrossQueues(moveAppRequest);
|
|
|
+
|
|
|
+ // move as the owner queue not in the acl
|
|
|
+ moveAppRequest = MoveApplicationAcrossQueuesRequest.newInstance(
|
|
|
+ applicationId, "not_allowed");
|
|
|
+
|
|
|
+ try {
|
|
|
+ rmService.moveApplicationAcrossQueues(moveAppRequest);
|
|
|
+ Assert.fail("The request should fail with an AccessControlException");
|
|
|
+ } catch (YarnException rex) {
|
|
|
+ Assert.assertTrue("AccessControlException is expected",
|
|
|
+ rex.getCause() instanceof AccessControlException);
|
|
|
+ }
|
|
|
+
|
|
|
+ // ACL is owned by "moveuser", move is performed as a different user
|
|
|
+ aclUGI = UserGroupInformation.createUserForTesting("moveuser",
|
|
|
+ new String[]{});
|
|
|
+ queueACLsManager = getQueueAclManager("move_queue",
|
|
|
+ QueueACL.SUBMIT_APPLICATIONS, aclUGI);
|
|
|
+ appAclsManager = getAppAclManager();
|
|
|
+ ClientRMService rmService2 =
|
|
|
+ createClientRMServiceForMoveApplicationRequest(applicationId,
|
|
|
+ aclUGI.getShortUserName(), appAclsManager, queueACLsManager);
|
|
|
+
|
|
|
+ // access to the queue not OK: user not allowed in this queue
|
|
|
+ MoveApplicationAcrossQueuesRequest moveAppRequest2 =
|
|
|
+ MoveApplicationAcrossQueuesRequest.
|
|
|
+ newInstance(applicationId, "move_queue");
|
|
|
+ try {
|
|
|
+ rmService2.moveApplicationAcrossQueues(moveAppRequest2);
|
|
|
+ Assert.fail("The request should fail with an AccessControlException");
|
|
|
+ } catch (YarnException rex) {
|
|
|
+ Assert.assertTrue("AccessControlException is expected",
|
|
|
+ rex.getCause() instanceof AccessControlException);
|
|
|
+ }
|
|
|
+
|
|
|
+ // execute the move as the acl owner
|
|
|
+ // access to the queue OK: user allowed in this queue
|
|
|
+ aclUGI.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
+ @Override
|
|
|
+ public Object run() throws Exception {
|
|
|
+ return rmService2.moveApplicationAcrossQueues(moveAppRequest2);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMoveApplicationAdminTargetQueue() throws Exception {
|
|
|
+ ApplicationId applicationId = getApplicationId(1);
|
|
|
+ UserGroupInformation aclUGI = UserGroupInformation.getCurrentUser();
|
|
|
+ QueueACLsManager queueAclsManager = getQueueAclManager("allowed_queue",
|
|
|
+ QueueACL.ADMINISTER_QUEUE, aclUGI);
|
|
|
+ ApplicationACLsManager appAclsManager = getAppAclManager();
|
|
|
+ ClientRMService rmService =
|
|
|
+ createClientRMServiceForMoveApplicationRequest(applicationId,
|
|
|
+ aclUGI.getShortUserName(), appAclsManager, queueAclsManager);
|
|
|
+
|
|
|
+ // user is admin move to queue in acl
|
|
|
+ MoveApplicationAcrossQueuesRequest moveAppRequest =
|
|
|
+ MoveApplicationAcrossQueuesRequest.newInstance(applicationId,
|
|
|
+ "allowed_queue");
|
|
|
+ rmService.moveApplicationAcrossQueues(moveAppRequest);
|
|
|
+
|
|
|
+ // user is admin move to queue not in acl
|
|
|
+ moveAppRequest = MoveApplicationAcrossQueuesRequest.newInstance(
|
|
|
+ applicationId, "not_allowed");
|
|
|
+
|
|
|
+ try {
|
|
|
+ rmService.moveApplicationAcrossQueues(moveAppRequest);
|
|
|
+ Assert.fail("The request should fail with an AccessControlException");
|
|
|
+ } catch (YarnException rex) {
|
|
|
+ Assert.assertTrue("AccessControlException is expected",
|
|
|
+ rex.getCause() instanceof AccessControlException);
|
|
|
+ }
|
|
|
+
|
|
|
+ // ACL is owned by "moveuser", move is performed as a different user
|
|
|
+ aclUGI = UserGroupInformation.createUserForTesting("moveuser",
|
|
|
+ new String[]{});
|
|
|
+ queueAclsManager = getQueueAclManager("move_queue",
|
|
|
+ QueueACL.ADMINISTER_QUEUE, aclUGI);
|
|
|
+ appAclsManager = getAppAclManager();
|
|
|
+ ClientRMService rmService2 =
|
|
|
+ createClientRMServiceForMoveApplicationRequest(applicationId,
|
|
|
+ aclUGI.getShortUserName(), appAclsManager, queueAclsManager);
|
|
|
+
|
|
|
+ // no access to this queue
|
|
|
+ MoveApplicationAcrossQueuesRequest moveAppRequest2 =
|
|
|
+ MoveApplicationAcrossQueuesRequest.
|
|
|
+ newInstance(applicationId, "move_queue");
|
|
|
+
|
|
|
+ try {
|
|
|
+ rmService2.moveApplicationAcrossQueues(moveAppRequest2);
|
|
|
+ Assert.fail("The request should fail with an AccessControlException");
|
|
|
+ } catch (YarnException rex) {
|
|
|
+ Assert.assertTrue("AccessControlException is expected",
|
|
|
+ rex.getCause() instanceof AccessControlException);
|
|
|
+ }
|
|
|
+
|
|
|
+ // execute the move as the acl owner
|
|
|
+ // access to the queue OK: user allowed in this queue
|
|
|
+ aclUGI.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
+ @Override
|
|
|
+ public Object run() throws Exception {
|
|
|
+ return rmService2.moveApplicationAcrossQueues(moveAppRequest2);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test (expected = YarnException.class)
|
|
|
+ public void testNonExistingQueue() throws Exception {
|
|
|
+ ApplicationId applicationId = getApplicationId(1);
|
|
|
+ UserGroupInformation aclUGI = UserGroupInformation.getCurrentUser();
|
|
|
+ QueueACLsManager queueAclsManager = getQueueAclManager();
|
|
|
+ ApplicationACLsManager appAclsManager = getAppAclManager();
|
|
|
+ ClientRMService rmService =
|
|
|
+ createClientRMServiceForMoveApplicationRequest(applicationId,
|
|
|
+ aclUGI.getShortUserName(), appAclsManager, queueAclsManager);
|
|
|
+
|
|
|
+ MoveApplicationAcrossQueuesRequest moveAppRequest =
|
|
|
+ MoveApplicationAcrossQueuesRequest.newInstance(applicationId,
|
|
|
+ "unknown_queue");
|
|
|
+ rmService.moveApplicationAcrossQueues(moveAppRequest);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create an instance of ClientRMService for testing
|
|
|
+ * moveApplicationAcrossQueues requests.
|
|
|
+ * @param applicationId the application
|
|
|
+ * @return ClientRMService
|
|
|
+ */
|
|
|
+ private ClientRMService createClientRMServiceForMoveApplicationRequest(
|
|
|
+ ApplicationId applicationId, String appOwner,
|
|
|
+ ApplicationACLsManager appAclsManager, QueueACLsManager queueAclsManager)
|
|
|
+ throws IOException {
|
|
|
+ RMApp app = mock(RMApp.class);
|
|
|
+ when(app.getUser()).thenReturn(appOwner);
|
|
|
+ when(app.getState()).thenReturn(RMAppState.RUNNING);
|
|
|
+ ConcurrentHashMap<ApplicationId, RMApp> apps = new ConcurrentHashMap<>();
|
|
|
+ apps.put(applicationId, app);
|
|
|
+
|
|
|
+ RMContext rmContext = mock(RMContext.class);
|
|
|
+ when(rmContext.getRMApps()).thenReturn(apps);
|
|
|
+
|
|
|
+ RMAppManager rmAppManager = mock(RMAppManager.class);
|
|
|
+ return new ClientRMService(rmContext, null, rmAppManager, appAclsManager,
|
|
|
+ queueAclsManager, null);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Plain application acl manager that always returns true.
|
|
|
+ * @return ApplicationACLsManager
|
|
|
+ */
|
|
|
+ private ApplicationACLsManager getAppAclManager() {
|
|
|
+ ApplicationACLsManager aclsManager = mock(ApplicationACLsManager.class);
|
|
|
+ when(aclsManager.checkAccess(
|
|
|
+ any(UserGroupInformation.class),
|
|
|
+ any(ApplicationAccessType.class),
|
|
|
+ any(String.class),
|
|
|
+ any(ApplicationId.class))).thenReturn(true);
|
|
|
+ return aclsManager;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Generate the Queue acl.
|
|
|
+ * @param allowedQueue the queue to allow the move to
|
|
|
+ * @param queueACL the acl to check: submit app or queue admin
|
|
|
+ * @param aclUser the user to check
|
|
|
+ * @return QueueACLsManager
|
|
|
+ */
|
|
|
+ private QueueACLsManager getQueueAclManager(String allowedQueue,
|
|
|
+ QueueACL queueACL, UserGroupInformation aclUser) throws IOException {
|
|
|
+ // ACL that checks the queue is allowed
|
|
|
+ QueueACLsManager queueACLsManager = mock(QueueACLsManager.class);
|
|
|
+ when(queueACLsManager.checkAccess(
|
|
|
+ any(UserGroupInformation.class),
|
|
|
+ any(QueueACL.class),
|
|
|
+ any(RMApp.class),
|
|
|
+ any(String.class),
|
|
|
+ anyListOf(String.class))).thenAnswer(new Answer<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean answer(InvocationOnMock invocationOnMock) {
|
|
|
+ final UserGroupInformation user =
|
|
|
+ (UserGroupInformation) invocationOnMock.getArguments()[0];
|
|
|
+ final QueueACL acl =
|
|
|
+ (QueueACL) invocationOnMock.getArguments()[1];
|
|
|
+ return (queueACL.equals(acl) &&
|
|
|
+ aclUser.getShortUserName().equals(user.getShortUserName()));
|
|
|
+ }
|
|
|
+ });
|
|
|
+
|
|
|
+ when(queueACLsManager.checkAccess(
|
|
|
+ any(UserGroupInformation.class),
|
|
|
+ any(QueueACL.class),
|
|
|
+ any(RMApp.class),
|
|
|
+ any(String.class),
|
|
|
+ anyListOf(String.class),
|
|
|
+ any(String.class))).thenAnswer(new Answer<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean answer(InvocationOnMock invocationOnMock) {
|
|
|
+ final UserGroupInformation user =
|
|
|
+ (UserGroupInformation) invocationOnMock.getArguments()[0];
|
|
|
+ final QueueACL acl = (QueueACL) invocationOnMock.getArguments()[1];
|
|
|
+ final String queue = (String) invocationOnMock.getArguments()[5];
|
|
|
+ return (allowedQueue.equals(queue) && queueACL.equals(acl) &&
|
|
|
+ aclUser.getShortUserName().equals(user.getShortUserName()));
|
|
|
+ }
|
|
|
+ });
|
|
|
+ return queueACLsManager;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * QueueACLsManager that always returns false when a target queue is passed
|
|
|
+ * in and true for other checks to simulate a missing queue.
|
|
|
+ * @return QueueACLsManager
|
|
|
+ */
|
|
|
+ private QueueACLsManager getQueueAclManager() {
|
|
|
+ QueueACLsManager queueACLsManager = mock(QueueACLsManager.class);
|
|
|
+ when(queueACLsManager.checkAccess(
|
|
|
+ any(UserGroupInformation.class),
|
|
|
+ any(QueueACL.class),
|
|
|
+ any(RMApp.class),
|
|
|
+ any(String.class),
|
|
|
+ anyListOf(String.class),
|
|
|
+ any(String.class))).thenReturn(false);
|
|
|
+ when(queueACLsManager.checkAccess(
|
|
|
+ any(UserGroupInformation.class),
|
|
|
+ any(QueueACL.class),
|
|
|
+ any(RMApp.class),
|
|
|
+ any(String.class),
|
|
|
+ anyListOf(String.class))).thenReturn(true);
|
|
|
+ return queueACLsManager;
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testGetQueueInfo() throws Exception {
|
|
|
YarnScheduler yarnScheduler = mock(YarnScheduler.class);
|