|
@@ -31,6 +31,7 @@ import static org.mockito.Mockito.verify;
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.PrintWriter;
|
|
import java.io.PrintWriter;
|
|
|
|
+import java.io.Serializable;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
import java.security.PrivilegedExceptionAction;
|
|
import java.security.PrivilegedExceptionAction;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
@@ -91,6 +92,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
|
@@ -110,6 +112,7 @@ import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerIn
|
|
import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
|
|
import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
|
|
+import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
@@ -457,7 +460,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
NMStateStoreService stateStore = new NMMemoryStateStoreService();
|
|
NMStateStoreService stateStore = new NMMemoryStateStoreService();
|
|
stateStore.init(conf);
|
|
stateStore.init(conf);
|
|
stateStore.start();
|
|
stateStore.start();
|
|
- Context context = createContext(conf, stateStore);
|
|
|
|
|
|
+ context = createContext(conf, stateStore);
|
|
ContainerManagerImpl cm = createContainerManager(context, delSrvc);
|
|
ContainerManagerImpl cm = createContainerManager(context, delSrvc);
|
|
((NMContext) context).setContainerManager(cm);
|
|
((NMContext) context).setContainerManager(cm);
|
|
cm.init(conf);
|
|
cm.init(conf);
|
|
@@ -467,55 +470,12 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
ApplicationAttemptId attemptId =
|
|
ApplicationAttemptId attemptId =
|
|
ApplicationAttemptId.newInstance(appId, 1);
|
|
ApplicationAttemptId.newInstance(appId, 1);
|
|
ContainerId cid = ContainerId.newContainerId(attemptId, 1);
|
|
ContainerId cid = ContainerId.newContainerId(attemptId, 1);
|
|
- Map<String, String> containerEnv = new HashMap<>();
|
|
|
|
- setFlowContext(containerEnv, "app_name1", appId);
|
|
|
|
- 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 = Collections.emptyMap();
|
|
|
|
- File tmpDir = new File("target",
|
|
|
|
- this.getClass().getSimpleName() + "-tmpDir");
|
|
|
|
- File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
|
|
|
|
- PrintWriter fileWriter = new PrintWriter(scriptFile);
|
|
|
|
- if (Shell.WINDOWS) {
|
|
|
|
- fileWriter.println("@ping -n 100 127.0.0.1 >nul");
|
|
|
|
- } else {
|
|
|
|
- fileWriter.write("\numask 0");
|
|
|
|
- fileWriter.write("\nexec sleep 100");
|
|
|
|
- }
|
|
|
|
- fileWriter.close();
|
|
|
|
- FileContext localFS = FileContext.getLocalFSFileContext();
|
|
|
|
- URL resource_alpha =
|
|
|
|
- URL.fromPath(localFS
|
|
|
|
- .makeQualified(new Path(scriptFile.getAbsolutePath())));
|
|
|
|
- LocalResource rsrc_alpha = RecordFactoryProvider
|
|
|
|
- .getRecordFactory(null).newRecordInstance(LocalResource.class);
|
|
|
|
- rsrc_alpha.setResource(resource_alpha);
|
|
|
|
- rsrc_alpha.setSize(-1);
|
|
|
|
- rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
|
|
|
|
- rsrc_alpha.setType(LocalResourceType.FILE);
|
|
|
|
- rsrc_alpha.setTimestamp(scriptFile.lastModified());
|
|
|
|
- String destinationFile = "dest_file";
|
|
|
|
- Map<String, LocalResource> localResources = new HashMap<>();
|
|
|
|
- localResources.put(destinationFile, rsrc_alpha);
|
|
|
|
- List<String> commands =
|
|
|
|
- Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
|
|
|
- ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
|
|
|
|
- localResources, containerEnv, commands, serviceData,
|
|
|
|
- containerTokens, acls);
|
|
|
|
- StartContainersResponse startResponse = startContainer(
|
|
|
|
- context, cm, cid, clc, null);
|
|
|
|
- assertTrue(startResponse.getFailedRequests().isEmpty());
|
|
|
|
- assertEquals(1, context.getApplications().size());
|
|
|
|
|
|
+
|
|
|
|
+ commonLaunchContainer(appId, cid, cm);
|
|
|
|
+
|
|
Application app = context.getApplications().get(appId);
|
|
Application app = context.getApplications().get(appId);
|
|
assertNotNull(app);
|
|
assertNotNull(app);
|
|
- // make sure the container reaches RUNNING state
|
|
|
|
- waitForNMContainerState(cm, cid,
|
|
|
|
- org.apache.hadoop.yarn.server.nodemanager
|
|
|
|
- .containermanager.container.ContainerState.RUNNING);
|
|
|
|
|
|
+
|
|
Resource targetResource = Resource.newInstance(2048, 2);
|
|
Resource targetResource = Resource.newInstance(2048, 2);
|
|
ContainerUpdateResponse updateResponse =
|
|
ContainerUpdateResponse updateResponse =
|
|
updateContainers(context, cm, cid, targetResource);
|
|
updateContainers(context, cm, cid, targetResource);
|
|
@@ -538,6 +498,62 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
assertEquals(targetResource, containerStatus.getCapability());
|
|
assertEquals(targetResource, containerStatus.getCapability());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testResourceMappingRecoveryForContainer() throws Exception {
|
|
|
|
+ conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true);
|
|
|
|
+ conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true);
|
|
|
|
+ NMStateStoreService stateStore = new NMMemoryStateStoreService();
|
|
|
|
+ stateStore.init(conf);
|
|
|
|
+ stateStore.start();
|
|
|
|
+ context = createContext(conf, stateStore);
|
|
|
|
+ ContainerManagerImpl cm = createContainerManager(context, delSrvc);
|
|
|
|
+ ((NMContext) context).setContainerManager(cm);
|
|
|
|
+ cm.init(conf);
|
|
|
|
+ cm.start();
|
|
|
|
+
|
|
|
|
+ // add an application by starting a container
|
|
|
|
+ ApplicationId appId = ApplicationId.newInstance(0, 1);
|
|
|
|
+ ApplicationAttemptId attemptId =
|
|
|
|
+ ApplicationAttemptId.newInstance(appId, 1);
|
|
|
|
+ ContainerId cid = ContainerId.newContainerId(attemptId, 1);
|
|
|
|
+
|
|
|
|
+ commonLaunchContainer(appId, cid, cm);
|
|
|
|
+
|
|
|
|
+ Application app = context.getApplications().get(appId);
|
|
|
|
+ assertNotNull(app);
|
|
|
|
+
|
|
|
|
+ // store resource mapping of the container
|
|
|
|
+ List<Serializable> gpuResources =
|
|
|
|
+ Arrays.<Serializable>asList("1", "2", "3");
|
|
|
|
+ stateStore.storeAssignedResources(cid, "gpu", gpuResources);
|
|
|
|
+ List<Serializable> numaResources = Arrays.<Serializable>asList("numa1");
|
|
|
|
+ stateStore.storeAssignedResources(cid, "numa", numaResources);
|
|
|
|
+ List<Serializable> fpgaResources =
|
|
|
|
+ Arrays.<Serializable>asList("fpga1", "fpga2");
|
|
|
|
+ stateStore.storeAssignedResources(cid, "fpga", fpgaResources);
|
|
|
|
+
|
|
|
|
+ cm.stop();
|
|
|
|
+ context = createContext(conf, stateStore);
|
|
|
|
+ cm = createContainerManager(context);
|
|
|
|
+ ((NMContext) context).setContainerManager(cm);
|
|
|
|
+ cm.init(conf);
|
|
|
|
+ cm.start();
|
|
|
|
+ assertEquals(1, context.getApplications().size());
|
|
|
|
+ app = context.getApplications().get(appId);
|
|
|
|
+ assertNotNull(app);
|
|
|
|
+
|
|
|
|
+ Container nmContainer = context.getContainers().get(cid);
|
|
|
|
+ Assert.assertNotNull(nmContainer);
|
|
|
|
+ ResourceMappings resourceMappings = nmContainer.getResourceMappings();
|
|
|
|
+ List<Serializable> assignedResource = resourceMappings
|
|
|
|
+ .getAssignedResources("gpu");
|
|
|
|
+ Assert.assertTrue(assignedResource.equals(gpuResources));
|
|
|
|
+ Assert.assertTrue(
|
|
|
|
+ resourceMappings.getAssignedResources("numa").equals(numaResources));
|
|
|
|
+ Assert.assertTrue(
|
|
|
|
+ resourceMappings.getAssignedResources("fpga").equals(fpgaResources));
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testContainerCleanupOnShutdown() throws Exception {
|
|
public void testContainerCleanupOnShutdown() throws Exception {
|
|
ApplicationId appId = ApplicationId.newInstance(0, 1);
|
|
ApplicationId appId = ApplicationId.newInstance(0, 1);
|
|
@@ -610,6 +626,57 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|
verify(cm, never()).handle(isA(CMgrCompletedAppsEvent.class));
|
|
verify(cm, never()).handle(isA(CMgrCompletedAppsEvent.class));
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void commonLaunchContainer(ApplicationId appId, ContainerId cid,
|
|
|
|
+ ContainerManagerImpl cm) throws Exception {
|
|
|
|
+ Map<String, String> containerEnv = new HashMap<>();
|
|
|
|
+ setFlowContext(containerEnv, "app_name1", appId);
|
|
|
|
+ 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 = Collections.emptyMap();
|
|
|
|
+ File tmpDir = new File("target",
|
|
|
|
+ this.getClass().getSimpleName() + "-tmpDir");
|
|
|
|
+ File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
|
|
|
|
+ PrintWriter fileWriter = new PrintWriter(scriptFile);
|
|
|
|
+ if (Shell.WINDOWS) {
|
|
|
|
+ fileWriter.println("@ping -n 100 127.0.0.1 >nul");
|
|
|
|
+ } else {
|
|
|
|
+ fileWriter.write("\numask 0");
|
|
|
|
+ fileWriter.write("\nexec sleep 100");
|
|
|
|
+ }
|
|
|
|
+ fileWriter.close();
|
|
|
|
+ FileContext localFS = FileContext.getLocalFSFileContext();
|
|
|
|
+ URL resource_alpha =
|
|
|
|
+ URL.fromPath(localFS
|
|
|
|
+ .makeQualified(new Path(scriptFile.getAbsolutePath())));
|
|
|
|
+ LocalResource rsrc_alpha = RecordFactoryProvider
|
|
|
|
+ .getRecordFactory(null).newRecordInstance(LocalResource.class);
|
|
|
|
+ rsrc_alpha.setResource(resource_alpha);
|
|
|
|
+ rsrc_alpha.setSize(-1);
|
|
|
|
+ rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
|
|
|
|
+ rsrc_alpha.setType(LocalResourceType.FILE);
|
|
|
|
+ rsrc_alpha.setTimestamp(scriptFile.lastModified());
|
|
|
|
+ String destinationFile = "dest_file";
|
|
|
|
+ Map<String, LocalResource> localResources = new HashMap<>();
|
|
|
|
+ localResources.put(destinationFile, rsrc_alpha);
|
|
|
|
+ List<String> commands =
|
|
|
|
+ Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
|
|
|
+ ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
|
|
|
|
+ localResources, containerEnv, commands, serviceData,
|
|
|
|
+ containerTokens, acls);
|
|
|
|
+ StartContainersResponse startResponse = startContainer(
|
|
|
|
+ context, cm, cid, clc, null);
|
|
|
|
+ assertTrue(startResponse.getFailedRequests().isEmpty());
|
|
|
|
+ assertEquals(1, context.getApplications().size());
|
|
|
|
+ // make sure the container reaches RUNNING state
|
|
|
|
+ waitForNMContainerState(cm, cid,
|
|
|
|
+ org.apache.hadoop.yarn.server.nodemanager
|
|
|
|
+ .containermanager.container.ContainerState.RUNNING);
|
|
|
|
+ }
|
|
|
|
+
|
|
private ContainerManagerImpl createContainerManager(Context context,
|
|
private ContainerManagerImpl createContainerManager(Context context,
|
|
DeletionService delSrvc) {
|
|
DeletionService delSrvc) {
|
|
return new ContainerManagerImpl(context, exec, delSrvc,
|
|
return new ContainerManagerImpl(context, exec, delSrvc,
|