|
@@ -38,9 +38,7 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
|
-import org.apache.hadoop.io.DataInputBuffer;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
-import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
@@ -49,7 +47,6 @@ import org.apache.hadoop.util.Shell;
|
|
|
import org.apache.hadoop.yarn.api.AMRMProtocol;
|
|
|
import org.apache.hadoop.yarn.api.ContainerManager;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
|
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
|
@@ -121,8 +118,8 @@ public class TestContainerManagerSecurity {
|
|
|
// Testing for malicious user
|
|
|
testMaliceUser();
|
|
|
|
|
|
- // Testing for unauthorized user
|
|
|
- testUnauthorizedUser();
|
|
|
+ // Testing for usage of expired tokens
|
|
|
+ testExpiredTokens();
|
|
|
|
|
|
} finally {
|
|
|
if (yarnCluster != null) {
|
|
@@ -184,6 +181,15 @@ public class TestContainerManagerSecurity {
|
|
|
resourceManager.getClientRMService().forceKillApplication(request);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * This tests a malice user getting a proper token but then messing with it by
|
|
|
+ * tampering with containerID/Resource etc.. His/her containers should be
|
|
|
+ * rejected.
|
|
|
+ *
|
|
|
+ * @throws IOException
|
|
|
+ * @throws InterruptedException
|
|
|
+ * @throws YarnRemoteException
|
|
|
+ */
|
|
|
private void testMaliceUser() throws IOException, InterruptedException,
|
|
|
YarnRemoteException {
|
|
|
|
|
@@ -205,30 +211,60 @@ public class TestContainerManagerSecurity {
|
|
|
appID);
|
|
|
|
|
|
// Now talk to the NM for launching the container with modified resource
|
|
|
- final ContainerId containerID = allocatedContainer.getId();
|
|
|
- UserGroupInformation maliceUser = UserGroupInformation
|
|
|
- .createRemoteUser(containerID.toString());
|
|
|
|
|
|
ContainerToken containerToken = allocatedContainer.getContainerToken();
|
|
|
- byte[] identifierBytes = containerToken.getIdentifier().array();
|
|
|
-
|
|
|
- DataInputBuffer di = new DataInputBuffer();
|
|
|
- di.reset(identifierBytes, identifierBytes.length);
|
|
|
-
|
|
|
- ContainerTokenIdentifier dummyIdentifier = new ContainerTokenIdentifier();
|
|
|
- dummyIdentifier.readFields(di);
|
|
|
+ ContainerTokenIdentifier originalContainerTokenId =
|
|
|
+ BuilderUtils.newContainerTokenIdentifier(containerToken);
|
|
|
|
|
|
// Malice user modifies the resource amount
|
|
|
Resource modifiedResource = BuilderUtils.newResource(2048, 1);
|
|
|
ContainerTokenIdentifier modifiedIdentifier =
|
|
|
- new ContainerTokenIdentifier(dummyIdentifier.getContainerID(),
|
|
|
- dummyIdentifier.getNmHostAddress(), "testUser", modifiedResource,
|
|
|
- Long.MAX_VALUE, dummyIdentifier.getMasterKeyId(),
|
|
|
+ new ContainerTokenIdentifier(originalContainerTokenId.getContainerID(),
|
|
|
+ originalContainerTokenId.getNmHostAddress(), "testUser",
|
|
|
+ modifiedResource, Long.MAX_VALUE,
|
|
|
+ originalContainerTokenId.getMasterKeyId(),
|
|
|
ResourceManager.clusterTimeStamp);
|
|
|
- Token<ContainerTokenIdentifier> modifiedToken = new Token<ContainerTokenIdentifier>(
|
|
|
- modifiedIdentifier.getBytes(), containerToken.getPassword().array(),
|
|
|
- new Text(containerToken.getKind()), new Text(containerToken
|
|
|
- .getService()));
|
|
|
+ Token<ContainerTokenIdentifier> modifiedToken =
|
|
|
+ new Token<ContainerTokenIdentifier>(modifiedIdentifier.getBytes(),
|
|
|
+ containerToken.getPassword().array(), new Text(
|
|
|
+ containerToken.getKind()), new Text(containerToken.getService()));
|
|
|
+ makeTamperedStartContainerCall(yarnRPC, allocatedContainer,
|
|
|
+ modifiedIdentifier, modifiedToken);
|
|
|
+
|
|
|
+ // Malice user modifies the container-Id
|
|
|
+ ContainerId newContainerId =
|
|
|
+ BuilderUtils.newContainerId(
|
|
|
+ BuilderUtils.newApplicationAttemptId(originalContainerTokenId
|
|
|
+ .getContainerID().getApplicationAttemptId().getApplicationId(), 1),
|
|
|
+ originalContainerTokenId.getContainerID().getId() + 42);
|
|
|
+ modifiedIdentifier =
|
|
|
+ new ContainerTokenIdentifier(newContainerId,
|
|
|
+ originalContainerTokenId.getNmHostAddress(), "testUser",
|
|
|
+ originalContainerTokenId.getResource(), Long.MAX_VALUE,
|
|
|
+ originalContainerTokenId.getMasterKeyId(),
|
|
|
+ ResourceManager.clusterTimeStamp);
|
|
|
+ modifiedToken =
|
|
|
+ new Token<ContainerTokenIdentifier>(modifiedIdentifier.getBytes(),
|
|
|
+ containerToken.getPassword().array(), new Text(
|
|
|
+ containerToken.getKind()), new Text(containerToken.getService()));
|
|
|
+ makeTamperedStartContainerCall(yarnRPC, allocatedContainer,
|
|
|
+ modifiedIdentifier, modifiedToken);
|
|
|
+
|
|
|
+ // Similarly messing with anything else will fail.
|
|
|
+
|
|
|
+ KillApplicationRequest request = Records
|
|
|
+ .newRecord(KillApplicationRequest.class);
|
|
|
+ request.setApplicationId(appID);
|
|
|
+ resourceManager.getClientRMService().forceKillApplication(request);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void makeTamperedStartContainerCall(final YarnRPC yarnRPC,
|
|
|
+ final Container allocatedContainer,
|
|
|
+ final ContainerTokenIdentifier modifiedIdentifier,
|
|
|
+ Token<ContainerTokenIdentifier> modifiedToken) {
|
|
|
+ final ContainerId containerID = allocatedContainer.getId();
|
|
|
+ UserGroupInformation maliceUser = UserGroupInformation
|
|
|
+ .createRemoteUser(containerID.toString());
|
|
|
maliceUser.addToken(modifiedToken);
|
|
|
maliceUser.doAs(new PrivilegedAction<Void>() {
|
|
|
@Override
|
|
@@ -239,11 +275,14 @@ public class TestContainerManagerSecurity {
|
|
|
conf);
|
|
|
|
|
|
LOG.info("Going to contact NM: ilLegal request");
|
|
|
- GetContainerStatusRequest request = recordFactory
|
|
|
- .newRecordInstance(GetContainerStatusRequest.class);
|
|
|
- request.setContainerId(containerID);
|
|
|
+ StartContainerRequest request =
|
|
|
+ Records.newRecord(StartContainerRequest.class);
|
|
|
try {
|
|
|
- client.getContainerStatus(request);
|
|
|
+ request.setContainerToken(allocatedContainer.getContainerToken());
|
|
|
+ ContainerLaunchContext context =
|
|
|
+ createContainerLaunchContextForTest(modifiedIdentifier);
|
|
|
+ request.setContainerLaunchContext(context);
|
|
|
+ client.startContainer(request);
|
|
|
fail("Connection initiation with illegally modified "
|
|
|
+ "tokens is expected to fail.");
|
|
|
} catch (YarnRemoteException e) {
|
|
@@ -263,14 +302,9 @@ public class TestContainerManagerSecurity {
|
|
|
return null;
|
|
|
}
|
|
|
});
|
|
|
-
|
|
|
- KillApplicationRequest request = Records
|
|
|
- .newRecord(KillApplicationRequest.class);
|
|
|
- request.setApplicationId(appID);
|
|
|
- resourceManager.getClientRMService().forceKillApplication(request);
|
|
|
}
|
|
|
|
|
|
- private void testUnauthorizedUser() throws IOException, InterruptedException,
|
|
|
+ private void testExpiredTokens() throws IOException, InterruptedException,
|
|
|
YarnRemoteException {
|
|
|
|
|
|
LOG.info("\n\nRunning test for malice user");
|
|
@@ -293,48 +327,12 @@ public class TestContainerManagerSecurity {
|
|
|
// Now talk to the NM for launching the container with modified containerID
|
|
|
final ContainerId containerID = allocatedContainer.getId();
|
|
|
|
|
|
- /////////// Test calls with illegal containerIDs and illegal Resources
|
|
|
- UserGroupInformation unauthorizedUser = UserGroupInformation
|
|
|
- .createRemoteUser(containerID.toString());
|
|
|
ContainerToken containerToken = allocatedContainer.getContainerToken();
|
|
|
-
|
|
|
- byte[] identifierBytes = containerToken.getIdentifier().array();
|
|
|
- DataInputBuffer di = new DataInputBuffer();
|
|
|
- di.reset(identifierBytes, identifierBytes.length);
|
|
|
- final ContainerTokenIdentifier tokenId = new ContainerTokenIdentifier();
|
|
|
- tokenId.readFields(di);
|
|
|
-
|
|
|
- Token<ContainerTokenIdentifier> token = new Token<ContainerTokenIdentifier>(
|
|
|
- identifierBytes, containerToken.getPassword().array(), new Text(
|
|
|
- containerToken.getKind()), new Text(containerToken.getService()));
|
|
|
-
|
|
|
- unauthorizedUser.addToken(token);
|
|
|
- ContainerManager client =
|
|
|
- unauthorizedUser.doAs(new PrivilegedAction<ContainerManager>() {
|
|
|
- @Override
|
|
|
- public ContainerManager run() {
|
|
|
- ContainerManager client = (ContainerManager) yarnRPC.getProxy(
|
|
|
- ContainerManager.class, NetUtils
|
|
|
- .createSocketAddr(allocatedContainer.getNodeId().toString()),
|
|
|
- conf);
|
|
|
-
|
|
|
- LOG.info("Going to contact NM: unauthorized request");
|
|
|
-
|
|
|
- callWithIllegalContainerID(client, tokenId, allocatedContainer);
|
|
|
- callWithIllegalResource(client, tokenId, allocatedContainer);
|
|
|
- // UserName is no longer sent using containerLaunchContext.
|
|
|
-// callWithIllegalUserName(client, tokenId, allocatedContainer);
|
|
|
-
|
|
|
- return client;
|
|
|
- }
|
|
|
- });
|
|
|
-
|
|
|
- // ///////// End of testing for illegal containerIDs, illegal Resources and
|
|
|
- // illegal users
|
|
|
+ final ContainerTokenIdentifier tokenId =
|
|
|
+ BuilderUtils.newContainerTokenIdentifier(containerToken);
|
|
|
|
|
|
/////////// Test calls with expired tokens
|
|
|
- RPC.stopProxy(client);
|
|
|
- unauthorizedUser = UserGroupInformation
|
|
|
+ UserGroupInformation unauthorizedUser = UserGroupInformation
|
|
|
.createRemoteUser(containerID.toString());
|
|
|
|
|
|
RMContainerTokenSecretManager containerTokenSecreteManager =
|
|
@@ -349,9 +347,10 @@ public class TestContainerManagerSecurity {
|
|
|
containerTokenSecreteManager.createPassword(
|
|
|
newTokenId);
|
|
|
// Create a valid token by using the key from the RM.
|
|
|
- token = new Token<ContainerTokenIdentifier>(
|
|
|
- newTokenId.getBytes(), passowrd, new Text(
|
|
|
- containerToken.getKind()), new Text(containerToken.getService()));
|
|
|
+ Token<ContainerTokenIdentifier> token =
|
|
|
+ new Token<ContainerTokenIdentifier>(newTokenId.getBytes(), passowrd,
|
|
|
+ new Text(containerToken.getKind()), new Text(
|
|
|
+ containerToken.getService()));
|
|
|
|
|
|
unauthorizedUser.addToken(token);
|
|
|
unauthorizedUser.doAs(new PrivilegedAction<Void>() {
|
|
@@ -369,7 +368,7 @@ public class TestContainerManagerSecurity {
|
|
|
request.setContainerLaunchContext(context);
|
|
|
allocatedContainer.setContainerToken(BuilderUtils.newContainerToken(
|
|
|
allocatedContainer.getNodeId(), passowrd, newTokenId));
|
|
|
- request.setContainer(allocatedContainer);
|
|
|
+ request.setContainerToken(allocatedContainer.getContainerToken());
|
|
|
|
|
|
//Calling startContainer with an expired token.
|
|
|
try {
|
|
@@ -524,93 +523,6 @@ public class TestContainerManagerSecurity {
|
|
|
return allocatedContainers.get(0);
|
|
|
}
|
|
|
|
|
|
- void callWithIllegalContainerID(ContainerManager client,
|
|
|
- ContainerTokenIdentifier tokenId, Container container) {
|
|
|
- StartContainerRequest request = recordFactory
|
|
|
- .newRecordInstance(StartContainerRequest.class);
|
|
|
- ContainerLaunchContext context =
|
|
|
- createContainerLaunchContextForTest(tokenId);
|
|
|
- ContainerId newContainerId = BuilderUtils.newContainerId(BuilderUtils
|
|
|
- .newApplicationAttemptId(tokenId.getContainerID()
|
|
|
- .getApplicationAttemptId().getApplicationId(), 1), 42);
|
|
|
- ContainerId oldContainerId = container.getId();
|
|
|
- try {
|
|
|
- container.setId(newContainerId);
|
|
|
- request.setContainer(container);
|
|
|
- request.setContainerLaunchContext(context);
|
|
|
- client.startContainer(request);
|
|
|
- fail("Connection initiation with unauthorized "
|
|
|
- + "access is expected to fail.");
|
|
|
- } catch (YarnRemoteException e) {
|
|
|
- LOG.info("Got exception : ", e);
|
|
|
- Assert.assertTrue(e.getMessage().contains(
|
|
|
- "Unauthorized request to start container. "
|
|
|
- + "\nExpected containerId: " + tokenId.getContainerID()
|
|
|
- + " Found: " + newContainerId.toString()));
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.info("Got IOException: ",e);
|
|
|
- fail("IOException is not expected.");
|
|
|
- } finally {
|
|
|
- container.setId(oldContainerId);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- void callWithIllegalResource(ContainerManager client,
|
|
|
- ContainerTokenIdentifier tokenId, Container container) {
|
|
|
- StartContainerRequest request = recordFactory
|
|
|
- .newRecordInstance(StartContainerRequest.class);
|
|
|
- // Authenticated but unauthorized, due to wrong resource
|
|
|
- ContainerLaunchContext context =
|
|
|
- createContainerLaunchContextForTest(tokenId);
|
|
|
- Resource rsrc = container.getResource();
|
|
|
- container.setResource(BuilderUtils.newResource(2048, 1));
|
|
|
- request.setContainerLaunchContext(context);
|
|
|
- request.setContainer(container);
|
|
|
- try {
|
|
|
- client.startContainer(request);
|
|
|
- fail("Connection initiation with unauthorized "
|
|
|
- + "access is expected to fail.");
|
|
|
- } catch (YarnRemoteException e) {
|
|
|
- LOG.info("Got exception : ", e);
|
|
|
- Assert.assertTrue(e.getMessage().contains(
|
|
|
- "Unauthorized request to start container. "));
|
|
|
- Assert.assertTrue(e.getMessage().contains(
|
|
|
- "\nExpected resource " + tokenId.getResource().toString()
|
|
|
- + " but found " + container.getResource().toString()));
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.info("Got IOException: ",e);
|
|
|
- fail("IOException is not expected.");
|
|
|
- }
|
|
|
- container.setResource(rsrc);
|
|
|
- }
|
|
|
-
|
|
|
- void callWithIllegalUserName(ContainerManager client,
|
|
|
- ContainerTokenIdentifier tokenId, Container container) {
|
|
|
- StartContainerRequest request = recordFactory
|
|
|
- .newRecordInstance(StartContainerRequest.class);
|
|
|
- // Authenticated but unauthorized, due to wrong resource
|
|
|
- ContainerLaunchContext context =
|
|
|
- createContainerLaunchContextForTest(tokenId);
|
|
|
- String user = "invalidUser";
|
|
|
- request.setContainerLaunchContext(context);
|
|
|
- request.setContainer(container);
|
|
|
- try {
|
|
|
- client.startContainer(request);
|
|
|
- fail("Connection initiation with unauthorized "
|
|
|
- + "access is expected to fail.");
|
|
|
- } catch (YarnRemoteException e) {
|
|
|
- LOG.info("Got exception : ", e);
|
|
|
- Assert.assertTrue(e.getMessage().contains(
|
|
|
- "Unauthorized request to start container. "));
|
|
|
- Assert.assertTrue(e.getMessage().contains(
|
|
|
- "Expected user-name " + tokenId.getApplicationSubmitter()
|
|
|
- + " but found " + user));
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.info("Got IOException: ",e);
|
|
|
- fail("IOException is not expected.");
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private ContainerLaunchContext createContainerLaunchContextForTest(
|
|
|
ContainerTokenIdentifier tokenId) {
|
|
|
ContainerLaunchContext context =
|