|
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.api.records.URL;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
|
|
@@ -89,13 +90,15 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
|
|
|
File shellFile = null;
|
|
|
File tempFile = null;
|
|
|
- String badSymlink = "foo@zz%_#*&!-+= bar()";
|
|
|
+ String badSymlink = Shell.WINDOWS ? "foo@zz_#!-+bar.cmd" :
|
|
|
+ "foo@zz%_#*&!-+= bar()";
|
|
|
File symLinkFile = null;
|
|
|
|
|
|
try {
|
|
|
- shellFile = new File(tmpDir, "hello.sh");
|
|
|
- tempFile = new File(tmpDir, "temp.sh");
|
|
|
- String timeoutCommand = "echo \"hello\"";
|
|
|
+ shellFile = Shell.appendScriptExtension(tmpDir, "hello");
|
|
|
+ tempFile = Shell.appendScriptExtension(tmpDir, "temp");
|
|
|
+ String timeoutCommand = Shell.WINDOWS ? "@echo \"hello\"" :
|
|
|
+ "echo \"hello\"";
|
|
|
PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));
|
|
|
shellFile.setExecutable(true);
|
|
|
writer.println(timeoutCommand);
|
|
@@ -110,7 +113,13 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
|
|
|
Map<String, String> env = new HashMap<String, String>();
|
|
|
List<String> commands = new ArrayList<String>();
|
|
|
- commands.add("/bin/sh ./\\\"" + badSymlink + "\\\"");
|
|
|
+ if (Shell.WINDOWS) {
|
|
|
+ commands.add("cmd");
|
|
|
+ commands.add("/c");
|
|
|
+ commands.add("\"" + badSymlink + "\"");
|
|
|
+ } else {
|
|
|
+ commands.add("/bin/sh ./\\\"" + badSymlink + "\\\"");
|
|
|
+ }
|
|
|
|
|
|
ContainerLaunch.writeLaunchEnv(fos, env, resources, commands);
|
|
|
fos.flush();
|
|
@@ -167,6 +176,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
recordFactory.newRecordInstance(ContainerId.class);
|
|
|
cId.setApplicationAttemptId(appAttemptId);
|
|
|
when(mockContainer.getId()).thenReturn(cId);
|
|
|
+
|
|
|
when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
|
|
|
when(mockContainer.getNodeHttpAddress()).thenReturn(
|
|
|
context.getNodeId().getHost() + ":12345");
|
|
@@ -185,19 +195,33 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
PrintWriter fileWriter = new PrintWriter(scriptFile);
|
|
|
File processStartFile =
|
|
|
new File(tmpDir, "env_vars.txt").getAbsoluteFile();
|
|
|
- fileWriter.write("\numask 0"); // So that start file is readable by the test
|
|
|
- fileWriter.write("\necho $" + Environment.CONTAINER_ID.name() + " > "
|
|
|
- + processStartFile);
|
|
|
- fileWriter.write("\necho $" + Environment.NM_HOST.name() + " >> "
|
|
|
- + processStartFile);
|
|
|
- fileWriter.write("\necho $" + Environment.NM_PORT.name() + " >> "
|
|
|
- + processStartFile);
|
|
|
- fileWriter.write("\necho $" + Environment.NM_HTTP_PORT.name() + " >> "
|
|
|
- + processStartFile);
|
|
|
- fileWriter.write("\necho $" + Environment.LOCAL_DIRS.name() + " >> "
|
|
|
- + processStartFile);
|
|
|
- fileWriter.write("\necho $$ >> " + processStartFile);
|
|
|
- fileWriter.write("\nexec sleep 100");
|
|
|
+ if (Shell.WINDOWS) {
|
|
|
+ fileWriter.println("@echo " + Environment.CONTAINER_ID.$() + "> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.println("@echo " + Environment.NM_HOST.$() + ">> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.println("@echo " + Environment.NM_PORT.$() + ">> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.println("@echo " + Environment.NM_HTTP_PORT.$() + ">> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.println("@echo " + Environment.LOCAL_DIRS.$() + ">> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.println("@ping -n 100 127.0.0.1 >nul");
|
|
|
+ } else {
|
|
|
+ fileWriter.write("\numask 0"); // So that start file is readable by the test
|
|
|
+ fileWriter.write("\necho $" + Environment.CONTAINER_ID.name() + " > "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.write("\necho $" + Environment.NM_HOST.name() + " >> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.write("\necho $" + Environment.NM_PORT.name() + " >> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.write("\necho $" + Environment.NM_HTTP_PORT.name() + " >> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.write("\necho $" + Environment.LOCAL_DIRS.name() + " >> "
|
|
|
+ + processStartFile);
|
|
|
+ fileWriter.write("\necho $$ >> " + processStartFile);
|
|
|
+ fileWriter.write("\nexec sleep 100");
|
|
|
+ }
|
|
|
fileWriter.close();
|
|
|
|
|
|
// upload the script file so that the container can run it
|
|
@@ -219,9 +243,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
|
|
|
// set up the rest of the container
|
|
|
containerLaunchContext.setUser(containerLaunchContext.getUser());
|
|
|
- List<String> commands = new ArrayList<String>();
|
|
|
- commands.add("/bin/bash");
|
|
|
- commands.add(scriptFile.getAbsolutePath());
|
|
|
+ List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
|
|
containerLaunchContext.setCommands(commands);
|
|
|
when(mockContainer.getResource()).thenReturn(
|
|
|
BuilderUtils.newResource(1024, 1));
|
|
@@ -282,12 +304,10 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
|
|
|
// Assert that the process is alive
|
|
|
Assert.assertTrue("Process is not alive!",
|
|
|
- exec.signalContainer(user,
|
|
|
- pid, Signal.NULL));
|
|
|
+ DefaultContainerExecutor.containerIsAlive(pid));
|
|
|
// Once more
|
|
|
Assert.assertTrue("Process is not alive!",
|
|
|
- exec.signalContainer(user,
|
|
|
- pid, Signal.NULL));
|
|
|
+ DefaultContainerExecutor.containerIsAlive(pid));
|
|
|
|
|
|
StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
|
|
|
stopRequest.setContainerId(cId);
|
|
@@ -301,38 +321,19 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
gcsRequest.setContainerId(cId);
|
|
|
ContainerStatus containerStatus =
|
|
|
containerManager.getContainerStatus(gcsRequest).getStatus();
|
|
|
- Assert.assertEquals(ExitCode.TERMINATED.getExitCode(),
|
|
|
- containerStatus.getExitStatus());
|
|
|
+ int expectedExitCode = Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
|
|
|
+ ExitCode.TERMINATED.getExitCode();
|
|
|
+ Assert.assertEquals(expectedExitCode, containerStatus.getExitStatus());
|
|
|
|
|
|
// Assert that the process is not alive anymore
|
|
|
Assert.assertFalse("Process is still alive!",
|
|
|
- exec.signalContainer(user,
|
|
|
- pid, Signal.NULL));
|
|
|
+ DefaultContainerExecutor.containerIsAlive(pid));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testDelayedKill() throws Exception {
|
|
|
containerManager.start();
|
|
|
|
|
|
- File processStartFile =
|
|
|
- new File(tmpDir, "pid.txt").getAbsoluteFile();
|
|
|
-
|
|
|
- // setup a script that can handle sigterm gracefully
|
|
|
- File scriptFile = new File(tmpDir, "testscript.sh");
|
|
|
- PrintWriter writer = new PrintWriter(new FileOutputStream(scriptFile));
|
|
|
- writer.println("#!/bin/bash\n\n");
|
|
|
- writer.println("echo \"Running testscript for delayed kill\"");
|
|
|
- writer.println("hello=\"Got SIGTERM\"");
|
|
|
- writer.println("umask 0");
|
|
|
- writer.println("trap \"echo $hello >> " + processStartFile + "\" SIGTERM");
|
|
|
- writer.println("echo \"Writing pid to start file\"");
|
|
|
- writer.println("echo $$ >> " + processStartFile);
|
|
|
- writer.println("while true; do\nsleep 1s;\ndone");
|
|
|
- writer.close();
|
|
|
- scriptFile.setExecutable(true);
|
|
|
-
|
|
|
- ContainerLaunchContext containerLaunchContext =
|
|
|
- recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
Container mockContainer = mock(Container.class);
|
|
|
// ////// Construct the Container-id
|
|
|
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
|
|
@@ -345,6 +346,33 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
ContainerId cId =
|
|
|
recordFactory.newRecordInstance(ContainerId.class);
|
|
|
cId.setApplicationAttemptId(appAttemptId);
|
|
|
+
|
|
|
+ File processStartFile =
|
|
|
+ new File(tmpDir, "pid.txt").getAbsoluteFile();
|
|
|
+
|
|
|
+ // setup a script that can handle sigterm gracefully
|
|
|
+ File scriptFile = Shell.appendScriptExtension(tmpDir, "testscript");
|
|
|
+ PrintWriter writer = new PrintWriter(new FileOutputStream(scriptFile));
|
|
|
+ if (Shell.WINDOWS) {
|
|
|
+ writer.println("@echo \"Running testscript for delayed kill\"");
|
|
|
+ writer.println("@echo \"Writing pid to start file\"");
|
|
|
+ writer.println("@echo " + cId + "> " + processStartFile);
|
|
|
+ writer.println("@ping -n 100 127.0.0.1 >nul");
|
|
|
+ } else {
|
|
|
+ writer.println("#!/bin/bash\n\n");
|
|
|
+ writer.println("echo \"Running testscript for delayed kill\"");
|
|
|
+ writer.println("hello=\"Got SIGTERM\"");
|
|
|
+ writer.println("umask 0");
|
|
|
+ writer.println("trap \"echo $hello >> " + processStartFile + "\" SIGTERM");
|
|
|
+ writer.println("echo \"Writing pid to start file\"");
|
|
|
+ writer.println("echo $$ >> " + processStartFile);
|
|
|
+ writer.println("while true; do\nsleep 1s;\ndone");
|
|
|
+ }
|
|
|
+ writer.close();
|
|
|
+ scriptFile.setExecutable(true);
|
|
|
+
|
|
|
+ ContainerLaunchContext containerLaunchContext =
|
|
|
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
when(mockContainer.getId()).thenReturn(cId);
|
|
|
when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
|
|
|
when(mockContainer.getNodeHttpAddress()).thenReturn(
|
|
@@ -372,8 +400,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
|
|
|
// set up the rest of the container
|
|
|
containerLaunchContext.setUser(containerLaunchContext.getUser());
|
|
|
- List<String> commands = new ArrayList<String>();
|
|
|
- commands.add(scriptFile.getAbsolutePath());
|
|
|
+ List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
|
|
|
containerLaunchContext.setCommands(commands);
|
|
|
when(mockContainer.getResource()).thenReturn(
|
|
|
BuilderUtils.newResource(1024, 1));
|
|
@@ -407,25 +434,32 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
Assert.assertEquals(ExitCode.FORCE_KILLED.getExitCode(),
|
|
|
containerStatus.getExitStatus());
|
|
|
|
|
|
- // Now verify the contents of the file
|
|
|
- // Script generates a message when it receives a sigterm
|
|
|
- // so we look for that
|
|
|
- BufferedReader reader =
|
|
|
- new BufferedReader(new FileReader(processStartFile));
|
|
|
-
|
|
|
- boolean foundSigTermMessage = false;
|
|
|
- while (true) {
|
|
|
- String line = reader.readLine();
|
|
|
- if (line == null) {
|
|
|
- break;
|
|
|
- }
|
|
|
- if (line.contains("SIGTERM")) {
|
|
|
- foundSigTermMessage = true;
|
|
|
- break;
|
|
|
+ // Now verify the contents of the file. Script generates a message when it
|
|
|
+ // receives a sigterm so we look for that. We cannot perform this check on
|
|
|
+ // Windows, because the process is not notified when killed by winutils.
|
|
|
+ // There is no way for the process to trap and respond. Instead, we can
|
|
|
+ // verify that the job object with ID matching container ID no longer exists.
|
|
|
+ if (Shell.WINDOWS) {
|
|
|
+ Assert.assertFalse("Process is still alive!",
|
|
|
+ DefaultContainerExecutor.containerIsAlive(cId.toString()));
|
|
|
+ } else {
|
|
|
+ BufferedReader reader =
|
|
|
+ new BufferedReader(new FileReader(processStartFile));
|
|
|
+
|
|
|
+ boolean foundSigTermMessage = false;
|
|
|
+ while (true) {
|
|
|
+ String line = reader.readLine();
|
|
|
+ if (line == null) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ if (line.contains("SIGTERM")) {
|
|
|
+ foundSigTermMessage = true;
|
|
|
+ break;
|
|
|
+ }
|
|
|
}
|
|
|
+ Assert.assertTrue("Did not find sigterm message", foundSigTermMessage);
|
|
|
+ reader.close();
|
|
|
}
|
|
|
- Assert.assertTrue("Did not find sigterm message", foundSigTermMessage);
|
|
|
- reader.close();
|
|
|
}
|
|
|
|
|
|
}
|