|
@@ -18,8 +18,6 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server;
|
|
|
|
|
|
-import static org.junit.Assert.fail;
|
|
|
-
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
@@ -31,6 +29,11 @@ import java.util.List;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+
|
|
|
+import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertNotNull;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertTrue;
|
|
|
+import static org.junit.jupiter.api.Assertions.fail;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.io.DataInputBuffer;
|
|
|
import org.apache.hadoop.minikdc.KerberosSecurityTestcase;
|
|
@@ -75,18 +78,15 @@ import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
|
-import org.junit.After;
|
|
|
-import org.junit.Assert;
|
|
|
-import org.junit.Before;
|
|
|
-import org.junit.Test;
|
|
|
-import org.junit.runner.RunWith;
|
|
|
-import org.junit.runners.Parameterized;
|
|
|
-import org.junit.runners.Parameterized.Parameters;
|
|
|
+import org.junit.jupiter.api.AfterEach;
|
|
|
+import org.junit.jupiter.api.BeforeEach;
|
|
|
+import org.junit.jupiter.params.ParameterizedTest;
|
|
|
+import org.junit.jupiter.params.provider.MethodSource;
|
|
|
+
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
|
-@RunWith(Parameterized.class)
|
|
|
public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
|
|
|
static Logger LOG = LoggerFactory.getLogger(TestContainerManagerSecurity.class);
|
|
@@ -94,29 +94,24 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
.getRecordFactory(null);
|
|
|
private static MiniYARNCluster yarnCluster;
|
|
|
private static final File testRootDir = new File("target",
|
|
|
- TestContainerManagerSecurity.class.getName() + "-root");
|
|
|
+ TestContainerManagerSecurity.class.getName() + "-root");
|
|
|
private static File httpSpnegoKeytabFile = new File(testRootDir,
|
|
|
- "httpSpnegoKeytabFile.keytab");
|
|
|
+ "httpSpnegoKeytabFile.keytab");
|
|
|
private static String httpSpnegoPrincipal = "HTTP/localhost@EXAMPLE.COM";
|
|
|
|
|
|
private Configuration conf;
|
|
|
|
|
|
- @Before
|
|
|
- public void setUp() throws Exception {
|
|
|
+ @BeforeEach
|
|
|
+ public void setup() throws Exception {
|
|
|
testRootDir.mkdirs();
|
|
|
httpSpnegoKeytabFile.deleteOnExit();
|
|
|
+ startMiniKdc();
|
|
|
getKdc().createPrincipal(httpSpnegoKeytabFile, httpSpnegoPrincipal);
|
|
|
- UserGroupInformation.setConfiguration(conf);
|
|
|
-
|
|
|
- yarnCluster =
|
|
|
- new MiniYARNCluster(TestContainerManagerSecurity.class.getName(), 1, 1,
|
|
|
- 1);
|
|
|
- yarnCluster.init(conf);
|
|
|
- yarnCluster.start();
|
|
|
}
|
|
|
-
|
|
|
- @After
|
|
|
+
|
|
|
+ @AfterEach
|
|
|
public void tearDown() {
|
|
|
+ stopMiniKdc();
|
|
|
if (yarnCluster != null) {
|
|
|
yarnCluster.stop();
|
|
|
yarnCluster = null;
|
|
@@ -130,48 +125,56 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
* and to give details in say an IDE. The second is the configuraiton
|
|
|
* object to use.
|
|
|
*/
|
|
|
- @Parameters(name = "{0}")
|
|
|
public static Collection<Object[]> configs() {
|
|
|
Configuration configurationWithoutSecurity = new Configuration();
|
|
|
configurationWithoutSecurity.set(
|
|
|
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "simple");
|
|
|
-
|
|
|
+
|
|
|
Configuration configurationWithSecurity = new Configuration();
|
|
|
configurationWithSecurity.set(
|
|
|
- CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
|
|
|
+ CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
|
|
|
configurationWithSecurity.set(
|
|
|
- YarnConfiguration.RM_WEBAPP_SPNEGO_USER_NAME_KEY, httpSpnegoPrincipal);
|
|
|
+ YarnConfiguration.RM_WEBAPP_SPNEGO_USER_NAME_KEY, httpSpnegoPrincipal);
|
|
|
configurationWithSecurity.set(
|
|
|
- YarnConfiguration.RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,
|
|
|
- httpSpnegoKeytabFile.getAbsolutePath());
|
|
|
+ YarnConfiguration.RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,
|
|
|
+ httpSpnegoKeytabFile.getAbsolutePath());
|
|
|
configurationWithSecurity.set(
|
|
|
- YarnConfiguration.NM_WEBAPP_SPNEGO_USER_NAME_KEY, httpSpnegoPrincipal);
|
|
|
+ YarnConfiguration.NM_WEBAPP_SPNEGO_USER_NAME_KEY, httpSpnegoPrincipal);
|
|
|
configurationWithSecurity.set(
|
|
|
- YarnConfiguration.NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,
|
|
|
- httpSpnegoKeytabFile.getAbsolutePath());
|
|
|
+ YarnConfiguration.NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,
|
|
|
+ httpSpnegoKeytabFile.getAbsolutePath());
|
|
|
|
|
|
- return Arrays.asList(new Object[][] {
|
|
|
+ return Arrays.asList(new Object[][]{
|
|
|
{"Simple", configurationWithoutSecurity},
|
|
|
{"Secure", configurationWithSecurity}});
|
|
|
}
|
|
|
-
|
|
|
- public TestContainerManagerSecurity(String name, Configuration conf) {
|
|
|
+
|
|
|
+ public void initTestContainerManagerSecurity(String name, Configuration conf) {
|
|
|
LOG.info("RUNNING TEST " + name);
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
|
+ yarnCluster =
|
|
|
+ new MiniYARNCluster(TestContainerManagerSecurity.class.getName(), 1, 1,
|
|
|
+ 1);
|
|
|
+ yarnCluster.init(conf);
|
|
|
+ yarnCluster.start();
|
|
|
conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 100000L);
|
|
|
this.conf = conf;
|
|
|
}
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testContainerManager() throws Exception {
|
|
|
-
|
|
|
- // TestNMTokens.
|
|
|
- testNMTokens(conf);
|
|
|
-
|
|
|
- // Testing for container token tampering
|
|
|
- testContainerToken(conf);
|
|
|
-
|
|
|
- // Testing for container token tampering with epoch
|
|
|
- testContainerTokenWithEpoch(conf);
|
|
|
+
|
|
|
+ @MethodSource("configs")
|
|
|
+ @ParameterizedTest(name = "{0}")
|
|
|
+ void testContainerManager(String name, Configuration conf) throws Exception {
|
|
|
+
|
|
|
+ initTestContainerManagerSecurity(name, conf);
|
|
|
+
|
|
|
+ // TestNMTokens.
|
|
|
+ testNMTokens(conf);
|
|
|
+
|
|
|
+ // Testing for container token tampering
|
|
|
+ testContainerToken(conf);
|
|
|
+
|
|
|
+ // Testing for container token tampering with epoch
|
|
|
+ testContainerTokenWithEpoch(conf);
|
|
|
|
|
|
}
|
|
|
|
|
@@ -182,21 +185,21 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
private void testNMTokens(Configuration testConf) throws Exception {
|
|
|
NMTokenSecretManagerInRM nmTokenSecretManagerRM =
|
|
|
yarnCluster.getResourceManager().getRMContext()
|
|
|
- .getNMTokenSecretManager();
|
|
|
+ .getNMTokenSecretManager();
|
|
|
NMTokenSecretManagerInNM nmTokenSecretManagerNM =
|
|
|
yarnCluster.getNodeManager(0).getNMContext().getNMTokenSecretManager();
|
|
|
RMContainerTokenSecretManager containerTokenSecretManager =
|
|
|
yarnCluster.getResourceManager().getRMContext().
|
|
|
getContainerTokenSecretManager();
|
|
|
-
|
|
|
+
|
|
|
NodeManager nm = yarnCluster.getNodeManager(0);
|
|
|
-
|
|
|
+
|
|
|
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerNM);
|
|
|
-
|
|
|
+
|
|
|
// Both id should be equal.
|
|
|
- Assert.assertEquals(nmTokenSecretManagerNM.getCurrentKey().getKeyId(),
|
|
|
+ assertEquals(nmTokenSecretManagerNM.getCurrentKey().getKeyId(),
|
|
|
nmTokenSecretManagerRM.getCurrentKey().getKeyId());
|
|
|
-
|
|
|
+
|
|
|
/*
|
|
|
* Below cases should be tested.
|
|
|
* 1) If Invalid NMToken is used then it should be rejected.
|
|
@@ -225,25 +228,25 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
yarnCluster.getResourceManager().getRMContext().getRMApps().put(appId, m);
|
|
|
ApplicationAttemptId validAppAttemptId =
|
|
|
ApplicationAttemptId.newInstance(appId, 1);
|
|
|
-
|
|
|
+
|
|
|
ContainerId validContainerId =
|
|
|
ContainerId.newContainerId(validAppAttemptId, 0);
|
|
|
-
|
|
|
+
|
|
|
NodeId validNode = yarnCluster.getNodeManager(0).getNMContext().getNodeId();
|
|
|
NodeId invalidNode = NodeId.newInstance("InvalidHost", 1234);
|
|
|
|
|
|
-
|
|
|
+
|
|
|
org.apache.hadoop.yarn.api.records.Token validNMToken =
|
|
|
nmTokenSecretManagerRM.createNMToken(validAppAttemptId, validNode, user);
|
|
|
-
|
|
|
+
|
|
|
org.apache.hadoop.yarn.api.records.Token validContainerToken =
|
|
|
containerTokenSecretManager.createContainerToken(validContainerId,
|
|
|
0, validNode, user, r, Priority.newInstance(10), 1234);
|
|
|
ContainerTokenIdentifier identifier =
|
|
|
BuilderUtils.newContainerTokenIdentifier(validContainerToken);
|
|
|
- Assert.assertEquals(Priority.newInstance(10), identifier.getPriority());
|
|
|
- Assert.assertEquals(1234, identifier.getCreationTime());
|
|
|
-
|
|
|
+ assertEquals(Priority.newInstance(10), identifier.getPriority());
|
|
|
+ assertEquals(1234, identifier.getCreationTime());
|
|
|
+
|
|
|
StringBuilder sb;
|
|
|
// testInvalidNMToken ... creating NMToken using different secret manager.
|
|
|
|
|
@@ -255,7 +258,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
// Making sure key id is different.
|
|
|
} while (tempManager.getCurrentKey().getKeyId() == nmTokenSecretManagerRM
|
|
|
.getCurrentKey().getKeyId());
|
|
|
-
|
|
|
+
|
|
|
// Testing that NM rejects the requests when we don't send any token.
|
|
|
if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
sb = new StringBuilder("Client cannot authenticate via:[TOKEN]");
|
|
@@ -266,55 +269,55 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
}
|
|
|
String errorMsg = testStartContainer(rpc, validAppAttemptId, validNode,
|
|
|
validContainerToken, null, true);
|
|
|
- Assert.assertTrue("In calling " + validNode + " exception was '"
|
|
|
+ assertTrue(errorMsg.contains(sb.toString()), "In calling " + validNode + " exception was '"
|
|
|
+ errorMsg + "' but doesn't contain '"
|
|
|
- + sb.toString() + "'", errorMsg.contains(sb.toString()));
|
|
|
-
|
|
|
+ + sb.toString() + "'");
|
|
|
+
|
|
|
org.apache.hadoop.yarn.api.records.Token invalidNMToken =
|
|
|
tempManager.createNMToken(validAppAttemptId, validNode, user);
|
|
|
sb = new StringBuilder("Given NMToken for application : ");
|
|
|
sb.append(validAppAttemptId.toString())
|
|
|
- .append(" seems to have been generated illegally.");
|
|
|
- Assert.assertTrue(sb.toString().contains(
|
|
|
+ .append(" seems to have been generated illegally.");
|
|
|
+ assertTrue(sb.toString().contains(
|
|
|
testStartContainer(rpc, validAppAttemptId, validNode,
|
|
|
validContainerToken, invalidNMToken, true)));
|
|
|
-
|
|
|
+
|
|
|
// valid NMToken but belonging to other node
|
|
|
invalidNMToken =
|
|
|
nmTokenSecretManagerRM.createNMToken(validAppAttemptId, invalidNode,
|
|
|
user);
|
|
|
sb = new StringBuilder("Given NMToken for application : ");
|
|
|
sb.append(validAppAttemptId)
|
|
|
- .append(" is not valid for current node manager.expected : ")
|
|
|
- .append(validNode.toString())
|
|
|
- .append(" found : ").append(invalidNode.toString());
|
|
|
- Assert.assertTrue(sb.toString().contains(
|
|
|
+ .append(" is not valid for current node manager.expected : ")
|
|
|
+ .append(validNode.toString())
|
|
|
+ .append(" found : ").append(invalidNode.toString());
|
|
|
+ assertTrue(sb.toString().contains(
|
|
|
testStartContainer(rpc, validAppAttemptId, validNode,
|
|
|
validContainerToken, invalidNMToken, true)));
|
|
|
-
|
|
|
+
|
|
|
// using correct tokens. nmtoken for app attempt should get saved.
|
|
|
testConf.setInt(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
|
|
|
4 * 60 * 1000);
|
|
|
validContainerToken =
|
|
|
containerTokenSecretManager.createContainerToken(validContainerId,
|
|
|
0, validNode, user, r, Priority.newInstance(0), 0);
|
|
|
- Assert.assertTrue(testStartContainer(rpc, validAppAttemptId, validNode,
|
|
|
- validContainerToken, validNMToken, false).isEmpty());
|
|
|
- Assert.assertTrue(nmTokenSecretManagerNM
|
|
|
+ assertTrue(testStartContainer(rpc, validAppAttemptId, validNode,
|
|
|
+ validContainerToken, validNMToken, false).isEmpty());
|
|
|
+ assertTrue(nmTokenSecretManagerNM
|
|
|
.isAppAttemptNMTokenKeyPresent(validAppAttemptId));
|
|
|
-
|
|
|
+
|
|
|
// using a new compatible version nmtoken, expect container can be started
|
|
|
// successfully.
|
|
|
ApplicationAttemptId validAppAttemptId2 =
|
|
|
ApplicationAttemptId.newInstance(appId, 2);
|
|
|
-
|
|
|
+
|
|
|
ContainerId validContainerId2 =
|
|
|
ContainerId.newContainerId(validAppAttemptId2, 0);
|
|
|
|
|
|
org.apache.hadoop.yarn.api.records.Token validContainerToken2 =
|
|
|
containerTokenSecretManager.createContainerToken(validContainerId2,
|
|
|
0, validNode, user, r, Priority.newInstance(0), 0);
|
|
|
-
|
|
|
+
|
|
|
org.apache.hadoop.yarn.api.records.Token validNMToken2 =
|
|
|
nmTokenSecretManagerRM.createNMToken(validAppAttemptId2, validNode, user);
|
|
|
// First, get a new NMTokenIdentifier.
|
|
@@ -323,43 +326,42 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
DataInputBuffer dib = new DataInputBuffer();
|
|
|
dib.reset(tokenIdentifierContent, tokenIdentifierContent.length);
|
|
|
newIdentifier.readFields(dib);
|
|
|
-
|
|
|
+
|
|
|
// Then, generate a new version NMTokenIdentifier (NMTokenIdentifierNewForTest)
|
|
|
// with additional field of message.
|
|
|
- NMTokenIdentifierNewForTest newVersionIdentifier =
|
|
|
+ NMTokenIdentifierNewForTest newVersionIdentifier =
|
|
|
new NMTokenIdentifierNewForTest(newIdentifier, "message");
|
|
|
-
|
|
|
+
|
|
|
// check new version NMTokenIdentifier has correct info.
|
|
|
- Assert.assertEquals("The ApplicationAttemptId is changed after set to " +
|
|
|
- "newVersionIdentifier", validAppAttemptId2.getAttemptId(),
|
|
|
+ assertEquals(validAppAttemptId2.getAttemptId(),
|
|
|
newVersionIdentifier.getApplicationAttemptId().getAttemptId()
|
|
|
- );
|
|
|
-
|
|
|
- Assert.assertEquals("The message is changed after set to newVersionIdentifier",
|
|
|
- "message", newVersionIdentifier.getMessage());
|
|
|
-
|
|
|
- Assert.assertEquals("The NodeId is changed after set to newVersionIdentifier",
|
|
|
- validNode, newVersionIdentifier.getNodeId());
|
|
|
-
|
|
|
+ ,
|
|
|
+ "The ApplicationAttemptId is changed after set to " +
|
|
|
+ "newVersionIdentifier");
|
|
|
+
|
|
|
+ assertEquals("message", newVersionIdentifier.getMessage(), "The message is changed after set to newVersionIdentifier");
|
|
|
+
|
|
|
+ assertEquals(validNode, newVersionIdentifier.getNodeId(), "The NodeId is changed after set to newVersionIdentifier");
|
|
|
+
|
|
|
// create new Token based on new version NMTokenIdentifier.
|
|
|
org.apache.hadoop.yarn.api.records.Token newVersionedNMToken =
|
|
|
BaseNMTokenSecretManager.newInstance(
|
|
|
- nmTokenSecretManagerRM.retrievePassword(newVersionIdentifier),
|
|
|
+ nmTokenSecretManagerRM.retrievePassword(newVersionIdentifier),
|
|
|
newVersionIdentifier);
|
|
|
-
|
|
|
+
|
|
|
// Verify startContainer is successful and no exception is thrown.
|
|
|
- Assert.assertTrue(testStartContainer(rpc, validAppAttemptId2, validNode,
|
|
|
+ assertTrue(testStartContainer(rpc, validAppAttemptId2, validNode,
|
|
|
validContainerToken2, newVersionedNMToken, false).isEmpty());
|
|
|
- Assert.assertTrue(nmTokenSecretManagerNM
|
|
|
+ assertTrue(nmTokenSecretManagerNM
|
|
|
.isAppAttemptNMTokenKeyPresent(validAppAttemptId2));
|
|
|
-
|
|
|
+
|
|
|
//Now lets wait till container finishes and is removed from node manager.
|
|
|
waitForContainerToFinishOnNM(validContainerId);
|
|
|
sb = new StringBuilder("Attempt to relaunch the same container with id ");
|
|
|
sb.append(validContainerId);
|
|
|
- Assert.assertTrue(testStartContainer(rpc, validAppAttemptId, validNode,
|
|
|
+ assertTrue(testStartContainer(rpc, validAppAttemptId, validNode,
|
|
|
validContainerToken, validNMToken, true).contains(sb.toString()));
|
|
|
-
|
|
|
+
|
|
|
// Container is removed from node manager's memory by this time.
|
|
|
// trying to stop the container. It should not throw any exception.
|
|
|
testStopContainer(rpc, validAppAttemptId, validNode, validContainerId,
|
|
@@ -370,25 +372,25 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
rollNMTokenMasterKey(nmTokenSecretManagerRM, nmTokenSecretManagerNM);
|
|
|
// Key rolled over once.. rolling over again
|
|
|
rollNMTokenMasterKey(nmTokenSecretManagerRM, nmTokenSecretManagerNM);
|
|
|
-
|
|
|
+
|
|
|
// trying get container status. Now saved nmToken should be used for
|
|
|
// authentication... It should complain saying container was recently
|
|
|
// stopped.
|
|
|
sb = new StringBuilder("Container ");
|
|
|
sb.append(validContainerId)
|
|
|
.append(" was recently stopped on node manager");
|
|
|
- Assert.assertTrue(testGetContainer(rpc, validAppAttemptId, validNode,
|
|
|
+ assertTrue(testGetContainer(rpc, validAppAttemptId, validNode,
|
|
|
validContainerId, validNMToken, true).contains(sb.toString()));
|
|
|
|
|
|
// Now lets remove the container from nm-memory
|
|
|
nm.getNodeStatusUpdater().clearFinishedContainersFromCache();
|
|
|
-
|
|
|
+
|
|
|
// This should fail as container is removed from recently tracked finished
|
|
|
// containers.
|
|
|
sb = new StringBuilder("Container ")
|
|
|
.append(validContainerId.toString())
|
|
|
.append(" is not handled by this NodeManager");
|
|
|
- Assert.assertTrue(testGetContainer(rpc, validAppAttemptId, validNode,
|
|
|
+ assertTrue(testGetContainer(rpc, validAppAttemptId, validNode,
|
|
|
validContainerId, validNMToken, false).contains(sb.toString()));
|
|
|
|
|
|
// using appAttempt-1 NMtoken for launching container for appAttempt-2
|
|
@@ -396,13 +398,13 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
ApplicationAttemptId attempt2 = ApplicationAttemptId.newInstance(appId, 2);
|
|
|
Token attempt1NMToken =
|
|
|
nmTokenSecretManagerRM
|
|
|
- .createNMToken(validAppAttemptId, validNode, user);
|
|
|
+ .createNMToken(validAppAttemptId, validNode, user);
|
|
|
org.apache.hadoop.yarn.api.records.Token newContainerToken =
|
|
|
containerTokenSecretManager.createContainerToken(
|
|
|
- ContainerId.newContainerId(attempt2, 1), 0, validNode, user, r,
|
|
|
+ ContainerId.newContainerId(attempt2, 1), 0, validNode, user, r,
|
|
|
Priority.newInstance(0), 0);
|
|
|
- Assert.assertTrue(testStartContainer(rpc, attempt2, validNode,
|
|
|
- newContainerToken, attempt1NMToken, false).isEmpty());
|
|
|
+ assertTrue(testStartContainer(rpc, attempt2, validNode,
|
|
|
+ newContainerToken, attempt1NMToken, false).isEmpty());
|
|
|
}
|
|
|
|
|
|
private void waitForContainerToFinishOnNM(ContainerId containerId)
|
|
@@ -419,7 +421,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
LOG.info("Waiting for " + containerId + " to get to state " +
|
|
|
ContainerState.COMPLETE);
|
|
|
GenericTestUtils.waitFor(() -> ContainerState.COMPLETE.equals(
|
|
|
- waitContainer.cloneAndGetContainerStatus().getState()),
|
|
|
+ waitContainer.cloneAndGetContainerStatus().getState()),
|
|
|
500, timeout);
|
|
|
} catch (TimeoutException te) {
|
|
|
LOG.error("TimeoutException", te);
|
|
@@ -433,7 +435,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
// Normally, Containers will be removed from NM context after they are
|
|
|
// explicitly acked by RM. Now, manually remove it for testing.
|
|
|
yarnCluster.getNodeManager(0).getNodeStatusUpdater()
|
|
|
- .addCompletedContainer(containerId);
|
|
|
+ .addCompletedContainer(containerId);
|
|
|
LOG.info("Removing container from NMContext, containerID = " + containerId);
|
|
|
nmContext.getContainers().remove(containerId);
|
|
|
}
|
|
@@ -458,16 +460,16 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
Thread.sleep(1000);
|
|
|
}
|
|
|
nmTokenSecretManagerRM.activateNextMasterKey();
|
|
|
- Assert.assertTrue((nmTokenSecretManagerNM.getCurrentKey().getKeyId()
|
|
|
+ assertTrue((nmTokenSecretManagerNM.getCurrentKey().getKeyId()
|
|
|
== nmTokenSecretManagerRM.getCurrentKey().getKeyId()));
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private String testStopContainer(YarnRPC rpc,
|
|
|
ApplicationAttemptId appAttemptId, NodeId nodeId,
|
|
|
ContainerId containerId, Token nmToken, boolean isExceptionExpected) {
|
|
|
try {
|
|
|
stopContainer(rpc, nmToken,
|
|
|
- Arrays.asList(new ContainerId[] {containerId}), appAttemptId,
|
|
|
+ Arrays.asList(new ContainerId[]{containerId}), appAttemptId,
|
|
|
nodeId);
|
|
|
if (isExceptionExpected) {
|
|
|
fail("Exception was expected!!");
|
|
@@ -505,8 +507,8 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
try {
|
|
|
startContainer(rpc, nmToken, containerToken, nodeId,
|
|
|
appAttemptId.toString());
|
|
|
- if (isExceptionExpected){
|
|
|
- fail("Exception was expected!!");
|
|
|
+ if (isExceptionExpected) {
|
|
|
+ fail("Exception was expected!!");
|
|
|
}
|
|
|
return "";
|
|
|
} catch (Exception e) {
|
|
@@ -514,7 +516,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
return e.getMessage();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private void stopContainer(YarnRPC rpc, Token nmToken,
|
|
|
List<ContainerId> containerId, ApplicationAttemptId appAttemptId,
|
|
|
NodeId nodeId) throws Exception {
|
|
@@ -537,13 +539,12 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private void
|
|
|
- getContainerStatus(YarnRPC rpc,
|
|
|
- org.apache.hadoop.yarn.api.records.Token nmToken,
|
|
|
- ContainerId containerId,
|
|
|
- ApplicationAttemptId appAttemptId, NodeId nodeId,
|
|
|
- boolean isExceptionExpected) throws Exception {
|
|
|
+
|
|
|
+ private void getContainerStatus(YarnRPC rpc,
|
|
|
+ org.apache.hadoop.yarn.api.records.Token nmToken,
|
|
|
+ ContainerId containerId,
|
|
|
+ ApplicationAttemptId appAttemptId, NodeId nodeId,
|
|
|
+ boolean isExceptionExpected) throws Exception {
|
|
|
List<ContainerId> containerIds = new ArrayList<ContainerId>();
|
|
|
containerIds.add(containerId);
|
|
|
GetContainerStatusesRequest request =
|
|
@@ -558,7 +559,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
if (statuses.getFailedRequests() != null
|
|
|
&& statuses.getFailedRequests().containsKey(containerId)) {
|
|
|
parseAndThrowException(statuses.getFailedRequests().get(containerId)
|
|
|
- .deSerialize());
|
|
|
+ .deSerialize());
|
|
|
}
|
|
|
} finally {
|
|
|
if (proxy != null) {
|
|
@@ -566,7 +567,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private void startContainer(final YarnRPC rpc,
|
|
|
org.apache.hadoop.yarn.api.records.Token nmToken,
|
|
|
org.apache.hadoop.yarn.api.records.Token containerToken,
|
|
@@ -584,7 +585,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
try {
|
|
|
proxy = getContainerManagementProtocolProxy(rpc, nmToken, nodeId, user);
|
|
|
StartContainersResponse response = proxy.startContainers(allRequests);
|
|
|
- for(SerializedException ex : response.getFailedRequests().values()){
|
|
|
+ for (SerializedException ex : response.getFailedRequests().values()) {
|
|
|
parseAndThrowException(ex.deSerialize());
|
|
|
}
|
|
|
} finally {
|
|
@@ -613,11 +614,11 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
final InetSocketAddress addr =
|
|
|
new InetSocketAddress(nodeId.getHost(), nodeId.getPort());
|
|
|
if (nmToken != null) {
|
|
|
- ugi.addToken(ConverterUtils.convertFromYarn(nmToken, addr));
|
|
|
+ ugi.addToken(ConverterUtils.convertFromYarn(nmToken, addr));
|
|
|
}
|
|
|
proxy =
|
|
|
NMProxy.createNMProxy(conf, ContainerManagementProtocol.class, ugi,
|
|
|
- rpc, addr);
|
|
|
+ rpc, addr);
|
|
|
return proxy;
|
|
|
}
|
|
|
|
|
@@ -642,7 +643,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
*/
|
|
|
NMTokenSecretManagerInRM nmTokenSecretManagerInRM =
|
|
|
yarnCluster.getResourceManager().getRMContext()
|
|
|
- .getNMTokenSecretManager();
|
|
|
+ .getNMTokenSecretManager();
|
|
|
ApplicationId appId = ApplicationId.newInstance(1, 1);
|
|
|
ApplicationAttemptId appAttemptId =
|
|
|
ApplicationAttemptId.newInstance(appId, 0);
|
|
@@ -651,46 +652,46 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
NMTokenSecretManagerInNM nmTokenSecretManagerInNM =
|
|
|
nm.getNMContext().getNMTokenSecretManager();
|
|
|
String user = "test";
|
|
|
-
|
|
|
+
|
|
|
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerInNM);
|
|
|
|
|
|
NodeId nodeId = nm.getNMContext().getNodeId();
|
|
|
-
|
|
|
+
|
|
|
// Both id should be equal.
|
|
|
- Assert.assertEquals(nmTokenSecretManagerInNM.getCurrentKey().getKeyId(),
|
|
|
+ assertEquals(nmTokenSecretManagerInNM.getCurrentKey().getKeyId(),
|
|
|
nmTokenSecretManagerInRM.getCurrentKey().getKeyId());
|
|
|
-
|
|
|
-
|
|
|
+
|
|
|
+
|
|
|
RMContainerTokenSecretManager containerTokenSecretManager =
|
|
|
yarnCluster.getResourceManager().getRMContext().
|
|
|
getContainerTokenSecretManager();
|
|
|
-
|
|
|
+
|
|
|
Resource r = Resource.newInstance(1230, 2);
|
|
|
-
|
|
|
- Token containerToken =
|
|
|
+
|
|
|
+ Token containerToken =
|
|
|
containerTokenSecretManager.createContainerToken(
|
|
|
cId, 0, nodeId, user, r, Priority.newInstance(0), 0);
|
|
|
-
|
|
|
- ContainerTokenIdentifier containerTokenIdentifier =
|
|
|
+
|
|
|
+ ContainerTokenIdentifier containerTokenIdentifier =
|
|
|
getContainerTokenIdentifierFromToken(containerToken);
|
|
|
-
|
|
|
+
|
|
|
// Verify new compatible version ContainerTokenIdentifier
|
|
|
// can work successfully.
|
|
|
- ContainerTokenIdentifierForTest newVersionTokenIdentifier =
|
|
|
+ ContainerTokenIdentifierForTest newVersionTokenIdentifier =
|
|
|
new ContainerTokenIdentifierForTest(containerTokenIdentifier,
|
|
|
"message");
|
|
|
- byte[] password =
|
|
|
+ byte[] password =
|
|
|
containerTokenSecretManager.createPassword(newVersionTokenIdentifier);
|
|
|
-
|
|
|
+
|
|
|
Token newContainerToken = BuilderUtils.newContainerToken(
|
|
|
nodeId, password, newVersionTokenIdentifier);
|
|
|
-
|
|
|
+
|
|
|
Token nmToken =
|
|
|
- nmTokenSecretManagerInRM.createNMToken(appAttemptId, nodeId, user);
|
|
|
+ nmTokenSecretManagerInRM.createNMToken(appAttemptId, nodeId, user);
|
|
|
YarnRPC rpc = YarnRPC.create(conf);
|
|
|
- Assert.assertTrue(testStartContainer(rpc, appAttemptId, nodeId,
|
|
|
+ assertTrue(testStartContainer(rpc, appAttemptId, nodeId,
|
|
|
newContainerToken, nmToken, false).isEmpty());
|
|
|
-
|
|
|
+
|
|
|
// Creating a tampered Container Token
|
|
|
RMContainerTokenSecretManager tamperedContainerTokenSecretManager =
|
|
|
new RMContainerTokenSecretManager(conf);
|
|
@@ -700,17 +701,17 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
tamperedContainerTokenSecretManager.activateNextMasterKey();
|
|
|
} while (containerTokenSecretManager.getCurrentKey().getKeyId()
|
|
|
== tamperedContainerTokenSecretManager.getCurrentKey().getKeyId());
|
|
|
-
|
|
|
+
|
|
|
ContainerId cId2 = ContainerId.newContainerId(appAttemptId, 1);
|
|
|
// Creating modified containerToken
|
|
|
Token containerToken2 =
|
|
|
tamperedContainerTokenSecretManager.createContainerToken(cId2, 0,
|
|
|
nodeId, user, r, Priority.newInstance(0), 0);
|
|
|
-
|
|
|
+
|
|
|
StringBuilder sb = new StringBuilder("Given Container ");
|
|
|
sb.append(cId2)
|
|
|
.append(" seems to have an illegally generated token.");
|
|
|
- Assert.assertTrue(testStartContainer(rpc, appAttemptId, nodeId,
|
|
|
+ assertTrue(testStartContainer(rpc, appAttemptId, nodeId,
|
|
|
containerToken2, nmToken, true).contains(sb.toString()));
|
|
|
}
|
|
|
|
|
@@ -754,7 +755,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
NodeId nodeId = nm.getNMContext().getNodeId();
|
|
|
|
|
|
// Both id should be equal.
|
|
|
- Assert.assertEquals(nmTokenSecretManagerInNM.getCurrentKey().getKeyId(),
|
|
|
+ assertEquals(nmTokenSecretManagerInNM.getCurrentKey().getKeyId(),
|
|
|
nmTokenSecretManagerInRM.getCurrentKey().getKeyId());
|
|
|
|
|
|
// Creating a normal Container Token
|
|
@@ -765,17 +766,17 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
Token containerToken =
|
|
|
containerTokenSecretManager.createContainerToken(cId, 0, nodeId, user,
|
|
|
r, Priority.newInstance(0), 0);
|
|
|
-
|
|
|
+
|
|
|
ContainerTokenIdentifier containerTokenIdentifier =
|
|
|
new ContainerTokenIdentifier();
|
|
|
byte[] tokenIdentifierContent = containerToken.getIdentifier().array();
|
|
|
DataInputBuffer dib = new DataInputBuffer();
|
|
|
dib.reset(tokenIdentifierContent, tokenIdentifierContent.length);
|
|
|
containerTokenIdentifier.readFields(dib);
|
|
|
-
|
|
|
-
|
|
|
- Assert.assertEquals(cId, containerTokenIdentifier.getContainerID());
|
|
|
- Assert.assertEquals(
|
|
|
+
|
|
|
+
|
|
|
+ assertEquals(cId, containerTokenIdentifier.getContainerID());
|
|
|
+ assertEquals(
|
|
|
cId.toString(), containerTokenIdentifier.getContainerID().toString());
|
|
|
|
|
|
Token nmToken =
|
|
@@ -791,10 +792,10 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
|
|
|
= getContainerManagementProtocolProxy(rpc, nmToken, nodeId, user);
|
|
|
GetContainerStatusesResponse res = proxy.getContainerStatuses(
|
|
|
GetContainerStatusesRequest.newInstance(containerIds));
|
|
|
- Assert.assertNotNull(res.getContainerStatuses().get(0));
|
|
|
- Assert.assertEquals(
|
|
|
+ assertNotNull(res.getContainerStatuses().get(0));
|
|
|
+ assertEquals(
|
|
|
cId, res.getContainerStatuses().get(0).getContainerId());
|
|
|
- Assert.assertEquals(cId.toString(),
|
|
|
+ assertEquals(cId.toString(),
|
|
|
res.getContainerStatuses().get(0).getContainerId().toString());
|
|
|
}
|
|
|
}
|