|
@@ -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.util.BuilderUtils;
|
|
@@ -88,13 +89,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);
|
|
@@ -109,7 +112,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();
|
|
@@ -145,16 +154,30 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
// this is a dirty hack - but should be ok for a unittest.
|
|
|
@SuppressWarnings({ "rawtypes", "unchecked" })
|
|
|
public static void setNewEnvironmentHack(Map<String, String> newenv) throws Exception {
|
|
|
- Class[] classes = Collections.class.getDeclaredClasses();
|
|
|
- Map<String, String> env = System.getenv();
|
|
|
- for (Class cl : classes) {
|
|
|
- if ("java.util.Collections$UnmodifiableMap".equals(cl.getName())) {
|
|
|
- Field field = cl.getDeclaredField("m");
|
|
|
- field.setAccessible(true);
|
|
|
- Object obj = field.get(env);
|
|
|
- Map<String, String> map = (Map<String, String>) obj;
|
|
|
- map.clear();
|
|
|
- map.putAll(newenv);
|
|
|
+ try {
|
|
|
+ Class<?> cl = Class.forName("java.lang.ProcessEnvironment");
|
|
|
+ Field field = cl.getDeclaredField("theEnvironment");
|
|
|
+ field.setAccessible(true);
|
|
|
+ Map<String, String> env = (Map<String, String>)field.get(null);
|
|
|
+ env.clear();
|
|
|
+ env.putAll(newenv);
|
|
|
+ Field ciField = cl.getDeclaredField("theCaseInsensitiveEnvironment");
|
|
|
+ ciField.setAccessible(true);
|
|
|
+ Map<String, String> cienv = (Map<String, String>)ciField.get(null);
|
|
|
+ cienv.clear();
|
|
|
+ cienv.putAll(newenv);
|
|
|
+ } catch (NoSuchFieldException e) {
|
|
|
+ Class[] classes = Collections.class.getDeclaredClasses();
|
|
|
+ Map<String, String> env = System.getenv();
|
|
|
+ for (Class cl : classes) {
|
|
|
+ if ("java.util.Collections$UnmodifiableMap".equals(cl.getName())) {
|
|
|
+ Field field = cl.getDeclaredField("m");
|
|
|
+ field.setAccessible(true);
|
|
|
+ Object obj = field.get(env);
|
|
|
+ Map<String, String> map = (Map<String, String>) obj;
|
|
|
+ map.clear();
|
|
|
+ map.putAll(newenv);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -172,22 +195,6 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
envWithDummy.put(Environment.MALLOC_ARENA_MAX.name(), "99");
|
|
|
setNewEnvironmentHack(envWithDummy);
|
|
|
|
|
|
- String malloc = System.getenv(Environment.MALLOC_ARENA_MAX.name());
|
|
|
- File scriptFile = new File(tmpDir, "scriptFile.sh");
|
|
|
- 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.MALLOC_ARENA_MAX.name() + " > " + processStartFile);
|
|
|
- fileWriter.write("\necho $$ >> " + processStartFile);
|
|
|
- fileWriter.write("\nexec sleep 100");
|
|
|
- fileWriter.close();
|
|
|
-
|
|
|
- assert(malloc != null && !"".equals(malloc));
|
|
|
-
|
|
|
- ContainerLaunchContext containerLaunchContext =
|
|
|
- recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
-
|
|
|
Container mockContainer = mock(Container.class);
|
|
|
// ////// Construct the Container-id
|
|
|
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
|
|
@@ -200,6 +207,30 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
|
|
|
ContainerId cId =
|
|
|
recordFactory.newRecordInstance(ContainerId.class);
|
|
|
cId.setApplicationAttemptId(appAttemptId);
|
|
|
+ String malloc = System.getenv(Environment.MALLOC_ARENA_MAX.name());
|
|
|
+ File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
|
|
|
+ PrintWriter fileWriter = new PrintWriter(scriptFile);
|
|
|
+ File processStartFile =
|
|
|
+ new File(tmpDir, "env_vars.txt").getAbsoluteFile();
|
|
|
+ if (Shell.WINDOWS) {
|
|
|
+ fileWriter.println("@echo " + Environment.MALLOC_ARENA_MAX.$() + "> " +
|
|
|
+ processStartFile);
|
|
|
+ fileWriter.println("@echo " + cId + ">> " + 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.MALLOC_ARENA_MAX.$() + " > " +
|
|
|
+ processStartFile);
|
|
|
+ fileWriter.write("\necho $$ >> " + processStartFile);
|
|
|
+ fileWriter.write("\nexec sleep 100");
|
|
|
+ }
|
|
|
+ fileWriter.close();
|
|
|
+
|
|
|
+ assert(malloc != null && !"".equals(malloc));
|
|
|
+
|
|
|
+ ContainerLaunchContext containerLaunchContext =
|
|
|
+ recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
|
+
|
|
|
when(mockContainer.getId()).thenReturn(cId);
|
|
|
|
|
|
containerLaunchContext.setUser(user);
|
|
@@ -223,9 +254,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));
|
|
@@ -255,12 +284,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);
|
|
@@ -274,38 +301,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);
|
|
@@ -318,6 +326,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);
|
|
|
|
|
|
containerLaunchContext.setUser(user);
|
|
@@ -341,8 +376,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));
|
|
@@ -376,25 +410,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();
|
|
|
}
|
|
|
|
|
|
}
|