|
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
|
+import static org.junit.Assert.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.mockito.Matchers.isA;
|
|
|
import static org.mockito.Mockito.mock;
|
|
@@ -74,6 +75,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
|
|
|
+import org.apache.hadoop.yarn.server.api.ContainerType;
|
|
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
|
|
import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
|
|
@@ -205,7 +207,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
|
"includePatternInRollingAggregation",
|
|
|
"excludePatternInRollingAggregation");
|
|
|
StartContainersResponse startResponse = startContainer(context, cm, cid,
|
|
|
- clc, logAggregationContext);
|
|
|
+ clc, logAggregationContext, ContainerType.TASK);
|
|
|
assertTrue(startResponse.getFailedRequests().isEmpty());
|
|
|
assertEquals(1, context.getApplications().size());
|
|
|
Application app = context.getApplications().get(appId);
|
|
@@ -342,7 +344,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
|
null, null);
|
|
|
|
|
|
StartContainersResponse startResponse = startContainer(context, cm, cid,
|
|
|
- clc, null);
|
|
|
+ clc, null, ContainerType.TASK);
|
|
|
assertTrue(startResponse.getFailedRequests().isEmpty());
|
|
|
assertEquals(1, context.getApplications().size());
|
|
|
Application app = context.getApplications().get(appId);
|
|
@@ -579,7 +581,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
|
cm.init(conf);
|
|
|
cm.start();
|
|
|
StartContainersResponse startResponse = startContainer(context, cm, cid,
|
|
|
- clc, logAggregationContext);
|
|
|
+ clc, logAggregationContext, ContainerType.TASK);
|
|
|
assertEquals(1, startResponse.getSuccessfullyStartedContainers().size());
|
|
|
cm.stop();
|
|
|
verify(cm).handle(isA(CMgrCompletedAppsEvent.class));
|
|
@@ -595,7 +597,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
|
cm.init(conf);
|
|
|
cm.start();
|
|
|
startResponse = startContainer(context, cm, cid,
|
|
|
- clc, logAggregationContext);
|
|
|
+ clc, logAggregationContext, ContainerType.TASK);
|
|
|
assertEquals(1, startResponse.getSuccessfullyStartedContainers().size());
|
|
|
cm.stop();
|
|
|
memStore.close();
|
|
@@ -612,7 +614,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
|
cm.init(conf);
|
|
|
cm.start();
|
|
|
startResponse = startContainer(context, cm, cid,
|
|
|
- clc, logAggregationContext);
|
|
|
+ clc, logAggregationContext, ContainerType.TASK);
|
|
|
assertEquals(1, startResponse.getSuccessfullyStartedContainers().size());
|
|
|
cm.stop();
|
|
|
memStore.close();
|
|
@@ -661,7 +663,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
|
localResources, containerEnv, commands, serviceData,
|
|
|
containerTokens, acls);
|
|
|
StartContainersResponse startResponse = startContainer(
|
|
|
- context, cm, cid, clc, null);
|
|
|
+ context, cm, cid, clc, null, ContainerType.TASK);
|
|
|
assertTrue(startResponse.getFailedRequests().isEmpty());
|
|
|
assertEquals(1, context.getApplications().size());
|
|
|
// make sure the container reaches RUNNING state
|
|
@@ -736,14 +738,15 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
|
|
|
|
private StartContainersResponse startContainer(Context context,
|
|
|
final ContainerManagerImpl cm, ContainerId cid,
|
|
|
- ContainerLaunchContext clc, LogAggregationContext logAggregationContext)
|
|
|
+ ContainerLaunchContext clc, LogAggregationContext logAggregationContext,
|
|
|
+ ContainerType containerType)
|
|
|
throws Exception {
|
|
|
UserGroupInformation user = UserGroupInformation.createRemoteUser(
|
|
|
cid.getApplicationAttemptId().toString());
|
|
|
StartContainerRequest scReq = StartContainerRequest.newInstance(
|
|
|
clc, TestContainerManager.createContainerToken(cid, 0,
|
|
|
context.getNodeId(), user.getShortUserName(),
|
|
|
- context.getContainerTokenSecretManager(), logAggregationContext));
|
|
|
+ context.getContainerTokenSecretManager(), logAggregationContext, containerType));
|
|
|
final List<StartContainerRequest> scReqList =
|
|
|
new ArrayList<StartContainerRequest>();
|
|
|
scReqList.add(scReq);
|
|
@@ -910,4 +913,91 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testApplicationRecoveryAfterFlowContextUpdated()
|
|
|
+ throws Exception {
|
|
|
+ conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true);
|
|
|
+ conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true);
|
|
|
+ conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
|
|
|
+ conf.set(YarnConfiguration.YARN_ADMIN_ACL, "yarn_admin_user");
|
|
|
+ NMStateStoreService stateStore = new NMMemoryStateStoreService();
|
|
|
+ stateStore.init(conf);
|
|
|
+ stateStore.start();
|
|
|
+ Context context = createContext(conf, stateStore);
|
|
|
+ ContainerManagerImpl cm = createContainerManager(context);
|
|
|
+ cm.init(conf);
|
|
|
+ cm.start();
|
|
|
+
|
|
|
+ // add an application by starting a container
|
|
|
+ String appName = "app_name1";
|
|
|
+ ApplicationId appId = ApplicationId.newInstance(0, 1);
|
|
|
+ ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
|
|
|
+
|
|
|
+ // create 1nd attempt container with containerId 2
|
|
|
+ ContainerId cid = ContainerId.newContainerId(attemptId, 2);
|
|
|
+ Map<String, LocalResource> localResources = Collections.emptyMap();
|
|
|
+ Map<String, String> containerEnv = new HashMap<>();
|
|
|
+
|
|
|
+ List<String> containerCmds = Collections.emptyList();
|
|
|
+ Map<String, ByteBuffer> serviceData = Collections.emptyMap();
|
|
|
+ Credentials containerCreds = new Credentials();
|
|
|
+ DataOutputBuffer dob = new DataOutputBuffer();
|
|
|
+ containerCreds.writeTokenStorageToStream(dob);
|
|
|
+ ByteBuffer containerTokens =
|
|
|
+ ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
|
|
|
+ Map<ApplicationAccessType, String> acls =
|
|
|
+ new HashMap<ApplicationAccessType, String>();
|
|
|
+ ContainerLaunchContext clc = ContainerLaunchContext
|
|
|
+ .newInstance(localResources, containerEnv, containerCmds, serviceData,
|
|
|
+ containerTokens, acls);
|
|
|
+ // create the logAggregationContext
|
|
|
+ LogAggregationContext logAggregationContext = LogAggregationContext
|
|
|
+ .newInstance("includePattern", "excludePattern",
|
|
|
+ "includePatternInRollingAggregation",
|
|
|
+ "excludePatternInRollingAggregation");
|
|
|
+
|
|
|
+ StartContainersResponse startResponse =
|
|
|
+ startContainer(context, cm, cid, clc, logAggregationContext,
|
|
|
+ ContainerType.TASK);
|
|
|
+ assertTrue(startResponse.getFailedRequests().isEmpty());
|
|
|
+ assertEquals(1, context.getApplications().size());
|
|
|
+ ApplicationImpl app =
|
|
|
+ (ApplicationImpl) context.getApplications().get(appId);
|
|
|
+ assertNotNull(app);
|
|
|
+ waitForAppState(app, ApplicationState.INITING);
|
|
|
+ assertNull(app.getFlowName());
|
|
|
+
|
|
|
+ // 2nd attempt
|
|
|
+ ApplicationAttemptId attemptId2 =
|
|
|
+ ApplicationAttemptId.newInstance(appId, 2);
|
|
|
+ // create 2nd attempt master container
|
|
|
+ ContainerId cid2 = ContainerId.newContainerId(attemptId, 1);
|
|
|
+ setFlowContext(containerEnv, appName, appId);
|
|
|
+ // once again create for updating launch context
|
|
|
+ clc = ContainerLaunchContext
|
|
|
+ .newInstance(localResources, containerEnv, containerCmds, serviceData,
|
|
|
+ containerTokens, acls);
|
|
|
+ // start container with container type AM.
|
|
|
+ startResponse =
|
|
|
+ startContainer(context, cm, cid2, clc, logAggregationContext,
|
|
|
+ ContainerType.APPLICATION_MASTER);
|
|
|
+ assertTrue(startResponse.getFailedRequests().isEmpty());
|
|
|
+ assertEquals(1, context.getApplications().size());
|
|
|
+ waitForAppState(app, ApplicationState.INITING);
|
|
|
+ assertEquals(appName, app.getFlowName());
|
|
|
+
|
|
|
+ // reset container manager and verify flow context information
|
|
|
+ cm.stop();
|
|
|
+ context = createContext(conf, stateStore);
|
|
|
+ cm = createContainerManager(context);
|
|
|
+ cm.init(conf);
|
|
|
+ cm.start();
|
|
|
+ assertEquals(1, context.getApplications().size());
|
|
|
+ app = (ApplicationImpl) context.getApplications().get(appId);
|
|
|
+ assertNotNull(app);
|
|
|
+ assertEquals(appName, app.getFlowName());
|
|
|
+ waitForAppState(app, ApplicationState.INITING);
|
|
|
+
|
|
|
+ cm.stop();
|
|
|
+ }
|
|
|
}
|