Bladeren bron

HADOOP-8562. Merge r1469667 for YARN-493 and HADOOP-9486 and r1469996 for HADOOP-9488.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1486231 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 jaren geleden
bovenliggende
commit
f0829a44e4
14 gewijzigde bestanden met toevoegingen van 340 en 205 verwijderingen
  1. 5 1
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 13 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ShellCommandFencer.java
  3. 70 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  4. 5 1
      hadoop-common-project/hadoop-common/src/main/winutils/task.c
  5. 13 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java
  6. 3 0
      hadoop-yarn-project/CHANGES.txt
  7. 0 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
  8. 17 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
  9. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  10. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
  11. 58 38
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
  12. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
  13. 45 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  14. 101 67
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

+ 5 - 1
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -341,7 +341,11 @@ Release 2.0.5-beta - UNRELEASED
 
     HADOOP-9490. LocalFileSystem#reportChecksumFailure not closing the 
     checksum file handle before rename. (Ivan Mitic via suresh)
-    
+
+    HADOOP-9488. FileUtil#createJarWithClassPath only substitutes environment
+    variables from current process environment/does not support overriding
+    when launching new process (Chris Nauroth via bikas)
+
 Release 2.0.4-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 13 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ShellCommandFencer.java

@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configured;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.util.Shell;
 
 /**
  * Fencing method that runs a shell command. It should be specified
@@ -33,8 +34,8 @@ import com.google.common.annotations.VisibleForTesting;
  * <code>
  *   shell(/path/to/my/script.sh arg1 arg2 ...)
  * </code><br>
- * The string between '(' and ')' is passed directly to a bash shell and
- * may not include any closing parentheses.<p>
+ * The string between '(' and ')' is passed directly to a bash shell
+ * (cmd.exe on Windows) and may not include any closing parentheses.<p>
  * 
  * The shell command will be run with an environment set up to contain
  * all of the current Hadoop configuration variables, with the '_' character 
@@ -58,11 +59,11 @@ public class ShellCommandFencer
 
   /** Prefix for target parameters added to the environment */
   private static final String TARGET_PREFIX = "target_";
-  
+
   @VisibleForTesting
   static Log LOG = LogFactory.getLog(
       ShellCommandFencer.class);
-  
+
   @Override
   public void checkArgs(String args) throws BadFencingConfigurationException {
     if (args == null || args.isEmpty()) {
@@ -74,8 +75,14 @@ public class ShellCommandFencer
 
   @Override
   public boolean tryFence(HAServiceTarget target, String cmd) {
-    ProcessBuilder builder = new ProcessBuilder(
-        "bash", "-e", "-c", cmd);
+    ProcessBuilder builder;
+
+    if (!Shell.WINDOWS) {
+      builder = new ProcessBuilder("bash", "-e", "-c", cmd);
+    } else {
+      builder = new ProcessBuilder("cmd.exe", "/c", cmd);
+    }
+
     setConfAsEnvVars(builder.environment());
     addTargetInfoAsEnvVars(target, builder.environment());
 

+ 70 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -123,6 +123,56 @@ abstract public class Shell {
                    : new String[] { "ln", "-s", target, link };
   }
 
+  /** Return a command for determining if process with specified pid is alive. */
+  public static String[] getCheckProcessIsAliveCommand(String pid) {
+    return Shell.WINDOWS ?
+      new String[] { Shell.WINUTILS, "task", "isAlive", pid } :
+      new String[] { "kill", "-0", isSetsidAvailable ? "-" + pid : pid };
+  }
+
+  /** Return a command to send a signal to a given pid */
+  public static String[] getSignalKillCommand(int code, String pid) {
+    return Shell.WINDOWS ? new String[] { Shell.WINUTILS, "task", "kill", pid } :
+      new String[] { "kill", "-" + code, isSetsidAvailable ? "-" + pid : pid };
+  }
+
+  /**
+   * Returns a File referencing a script with the given basename, inside the
+   * given parent directory.  The file extension is inferred by platform: ".cmd"
+   * on Windows, or ".sh" otherwise.
+   * 
+   * @param parent File parent directory
+   * @param basename String script file basename
+   * @return File referencing the script in the directory
+   */
+  public static File appendScriptExtension(File parent, String basename) {
+    return new File(parent, appendScriptExtension(basename));
+  }
+
+  /**
+   * Returns a script file name with the given basename.  The file extension is
+   * inferred by platform: ".cmd" on Windows, or ".sh" otherwise.
+   * 
+   * @param basename String script file basename
+   * @return String script file name
+   */
+  public static String appendScriptExtension(String basename) {
+    return basename + (WINDOWS ? ".cmd" : ".sh");
+  }
+
+  /**
+   * Returns a command to run the given script.  The script interpreter is
+   * inferred by platform: cmd on Windows or bash otherwise.
+   * 
+   * @param script File script to run
+   * @return String[] command to run the script
+   */
+  public static String[] getRunScriptCommand(File script) {
+    String absolutePath = script.getAbsolutePath();
+    return WINDOWS ? new String[] { "cmd", "/c", absolutePath } :
+      new String[] { "/bin/bash", absolutePath };
+  }
+
   /** a Unix command to set permission */
   public static final String SET_PERMISSION_COMMAND = "chmod";
   /** a Unix command to set owner */
@@ -243,6 +293,26 @@ abstract public class Shell {
     return winUtilsPath;
   }
 
+  public static final boolean isSetsidAvailable = isSetsidSupported();
+  private static boolean isSetsidSupported() {
+    if (Shell.WINDOWS) {
+      return false;
+    }
+    ShellCommandExecutor shexec = null;
+    boolean setsidSupported = true;
+    try {
+      String[] args = {"setsid", "bash", "-c", "echo $$"};
+      shexec = new ShellCommandExecutor(args);
+      shexec.execute();
+    } catch (IOException ioe) {
+      LOG.warn("setsid is not available on this machine. So not using it.");
+      setsidSupported = false;
+    } finally { // handle the exit code
+      LOG.info("setsid exited with exit code " + shexec.getExitCode());
+    }
+    return setsidSupported;
+  }
+
   /** Token separator regex used to parse Shell tool outputs */
   public static final String TOKEN_SEPARATOR_REGEX
                 = WINDOWS ? "[|\n\r]" : "[ \t\n\r\f]";

+ 5 - 1
hadoop-common-project/hadoop-common/src/main/winutils/task.c

@@ -24,6 +24,10 @@
 
 #define ERROR_TASK_NOT_ALIVE 1
 
+// This exit code for killed processes is compatible with Unix, where a killed
+// process exits with 128 + signal.  For SIGKILL, this would be 128 + 9 = 137.
+#define KILLED_PROCESS_EXIT_CODE 137
+
 // List of different task related command line options supported by
 // winutils.
 typedef enum TaskCommandOptionType
@@ -266,7 +270,7 @@ DWORD killTask(PCWSTR jobObjName)
     return err;
   }
 
-  if(TerminateJobObject(jobObject, 1) == 0)
+  if(TerminateJobObject(jobObject, KILLED_PROCESS_EXIT_CODE) == 0)
   {
     return GetLastError();
   }

+ 13 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java

@@ -24,6 +24,7 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Shell;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -33,7 +34,12 @@ import com.google.common.collect.Lists;
 public class TestNodeFencer {
 
   private HAServiceTarget MOCK_TARGET;
-  
+
+  // Fencer shell commands that always return true on Unix and Windows
+  // respectively. Lacking the POSIX 'true' command on Windows, we use
+  // the batch command 'rem'.
+  private static String FENCER_TRUE_COMMAND_UNIX = "shell(true)";
+  private static String FENCER_TRUE_COMMAND_WINDOWS = "shell(rem)";
 
   @Before
   public void clearMockState() {
@@ -48,6 +54,11 @@ public class TestNodeFencer {
         .when(MOCK_TARGET).getAddress();
   }
 
+  private static String getFencerTrueCommand() {
+    return Shell.WINDOWS ?
+        FENCER_TRUE_COMMAND_WINDOWS : FENCER_TRUE_COMMAND_UNIX;
+  }
+
   @Test
   public void testSingleFencer() throws BadFencingConfigurationException {
     NodeFencer fencer = setupFencer(
@@ -100,7 +111,7 @@ public class TestNodeFencer {
 
   @Test
   public void testShortNameShell() throws BadFencingConfigurationException {
-    NodeFencer fencer = setupFencer("shell(true)");
+    NodeFencer fencer = setupFencer(getFencerTrueCommand());
     assertTrue(fencer.fence(MOCK_TARGET));
   }
 

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -386,6 +386,9 @@ Release 2.0.5-beta - UNRELEASED
     classpath with new process's environment variables and localized resources
     (Chris Nauroth via bikas)
 
+    YARN-493. Fixed some shell related flaws in YARN on Windows. (Chris Nauroth
+    via vinodkv)
+
 Release 2.0.4-alpha - 2013-04-25 
 
   INCOMPATIBLE CHANGES

+ 0 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java

@@ -229,19 +229,6 @@ public abstract class ContainerExecutor implements Configurable {
     }
   }   
 
-  /** Return a command for determining if process with specified pid is alive. */
-  protected static String[] getCheckProcessIsAliveCommand(String pid) {
-    return Shell.WINDOWS ?
-      new String[] { Shell.WINUTILS, "task", "isAlive", pid } :
-      new String[] { "kill", "-0", pid };
-  }
-
-  /** Return a command to send a signal to a given pid */
-  protected static String[] getSignalKillCommand(int code, String pid) {
-    return Shell.WINDOWS ? new String[] { Shell.WINUTILS, "task", "kill", pid } :
-      new String[] { "kill", "-" + code, pid };
-  }
-
   /**
    * Is the container still active?
    * @param containerId
@@ -310,26 +297,6 @@ public abstract class ContainerExecutor implements Configurable {
     return pid;
   }
 
-  public static final boolean isSetsidAvailable = isSetsidSupported();
-  private static boolean isSetsidSupported() {
-    if (Shell.WINDOWS) {
-      return true;
-    }
-    ShellCommandExecutor shexec = null;
-    boolean setsidSupported = true;
-    try {
-      String[] args = {"setsid", "bash", "-c", "echo $$"};
-      shexec = new ShellCommandExecutor(args);
-      shexec.execute();
-    } catch (IOException ioe) {
-      LOG.warn("setsid is not available on this machine. So not using it.");
-      setsidSupported = false;
-    } finally { // handle the exit code
-      LOG.info("setsid exited with exit code " + shexec.getExitCode());
-    }
-    return setsidSupported;
-  }
-
   public static class DelayedProcessKiller extends Thread {
     private final String user;
     private final String pid;

+ 17 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java

@@ -50,6 +50,8 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 public class DefaultContainerExecutor extends ContainerExecutor {
 
   private static final Log LOG = LogFactory
@@ -237,8 +239,9 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     protected abstract void writeLocalWrapperScript(Path launchDst, Path pidFile,
         PrintStream pout);
 
-    protected LocalWrapperScriptBuilder(Path wrapperScriptPath) {
-      this.wrapperScriptPath = wrapperScriptPath;
+    protected LocalWrapperScriptBuilder(Path containerWorkDir) {
+      this.wrapperScriptPath = new Path(containerWorkDir,
+        Shell.appendScriptExtension("default_container_executor"));
     }
   }
 
@@ -246,7 +249,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       extends LocalWrapperScriptBuilder {
 
     public UnixLocalWrapperScriptBuilder(Path containerWorkDir) {
-      super(new Path(containerWorkDir, "default_container_executor.sh"));
+      super(containerWorkDir);
     }
 
     @Override
@@ -260,7 +263,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       pout.println();
       pout.println("echo $$ > " + pidFile.toString() + ".tmp");
       pout.println("/bin/mv -f " + pidFile.toString() + ".tmp " + pidFile);
-      String exec = ContainerExecutor.isSetsidAvailable? "exec setsid" : "exec";
+      String exec = Shell.isSetsidAvailable? "exec setsid" : "exec";
       pout.println(exec + " /bin/bash -c \"" +
         launchDst.toUri().getPath().toString() + "\"");
     }
@@ -274,7 +277,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     public WindowsLocalWrapperScriptBuilder(String containerIdStr,
         Path containerWorkDir) {
 
-      super(new Path(containerWorkDir, "default_container_executor.cmd"));
+      super(containerWorkDir);
       this.containerIdStr = containerIdStr;
     }
 
@@ -297,18 +300,15 @@ public class DefaultContainerExecutor extends ContainerExecutor {
   @Override
   public boolean signalContainer(String user, String pid, Signal signal)
       throws IOException {
-    final String sigpid = ContainerExecutor.isSetsidAvailable
-        ? "-" + pid
-        : pid;
-    LOG.debug("Sending signal " + signal.getValue() + " to pid " + sigpid
+    LOG.debug("Sending signal " + signal.getValue() + " to pid " + pid
         + " as user " + user);
-    if (!containerIsAlive(sigpid)) {
+    if (!containerIsAlive(pid)) {
       return false;
     }
     try {
-      killContainer(sigpid, signal);
+      killContainer(pid, signal);
     } catch (IOException e) {
-      if (!containerIsAlive(sigpid)) {
+      if (!containerIsAlive(pid)) {
         return false;
       }
       throw e;
@@ -322,9 +322,11 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * @param pid String pid
    * @return boolean true if the process is alive
    */
-  private boolean containerIsAlive(String pid) throws IOException {
+  @VisibleForTesting
+  public static boolean containerIsAlive(String pid) throws IOException {
     try {
-      new ShellCommandExecutor(getCheckProcessIsAliveCommand(pid)).execute();
+      new ShellCommandExecutor(Shell.getCheckProcessIsAliveCommand(pid))
+        .execute();
       // successful execution means process is alive
       return true;
     }
@@ -342,7 +344,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * (for logging).
    */
   private void killContainer(String pid, Signal signal) throws IOException {
-    new ShellCommandExecutor(getSignalKillCommand(signal.getValue(), pid))
+    new ShellCommandExecutor(Shell.getSignalKillCommand(signal.getValue(), pid))
       .execute();
   }
 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -73,8 +73,8 @@ public class ContainerLaunch implements Callable<Integer> {
 
   private static final Log LOG = LogFactory.getLog(ContainerLaunch.class);
 
-  public static final String CONTAINER_SCRIPT = Shell.WINDOWS ?
-    "launch_container.cmd" : "launch_container.sh";
+  public static final String CONTAINER_SCRIPT =
+    Shell.appendScriptExtension("launch_container");
   public static final String FINAL_CONTAINER_TOKENS_FILE = "container_tokens";
 
   private static final String PID_FILE_NAME_FMT = "%s.pid";

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java

@@ -89,7 +89,8 @@ public class TestNodeManagerResync {
     YarnConfiguration conf = createNMConfig();
     nm.init(conf);
     nm.start();
-    TestNodeManagerShutdown.startContainer(nm, localFS, tmpDir,
+    ContainerId cId = TestNodeManagerShutdown.createContainerId();
+    TestNodeManagerShutdown.startContainer(nm, cId, localFS, tmpDir,
       processStartFile);
 
     Assert.assertEquals(1, ((TestNodeManager1) nm).getNMRegistrationCount());
@@ -118,7 +119,8 @@ public class TestNodeManagerResync {
     nm.start();
 
     // Start the container in running state
-    TestNodeManagerShutdown.startContainer(nm, localFS, tmpDir,
+    ContainerId cId = TestNodeManagerShutdown.createContainerId();
+    TestNodeManagerShutdown.startContainer(nm, cId, localFS, tmpDir,
       processStartFile);
 
     nm.getNMDispatcher().getEventHandler()

+ 58 - 38
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java

@@ -19,14 +19,15 @@
 package org.apache.hadoop.yarn.server.nodemanager;
 
 import java.io.BufferedReader;
-import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
+import java.io.PrintWriter;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -40,6 +41,7 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ContainerManager;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -64,6 +66,9 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
+import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
@@ -84,6 +89,7 @@ public class TestNodeManagerShutdown {
       .getRecordFactory(null);
   static final String user = "nobody";
   private FileContext localFS;
+  private ContainerId cId;
 
   @Before
   public void setup() throws UnsupportedFileSystemException {
@@ -92,6 +98,9 @@ public class TestNodeManagerShutdown {
     logsDir.mkdirs();
     remoteLogsDir.mkdirs();
     nmLocalDir.mkdirs();
+
+    // Construct the Container-id
+    cId = createContainerId();
   }
   
   @After
@@ -105,7 +114,7 @@ public class TestNodeManagerShutdown {
     NodeManager nm = new TestNodeManager();
     nm.init(createNMConfig());
     nm.start();
-    startContainer(nm, localFS, tmpDir, processStartFile);
+    startContainer(nm, cId, localFS, tmpDir, processStartFile);
     
     final int MAX_TRIES=20;
     int numTries = 0;
@@ -118,38 +127,43 @@ public class TestNodeManagerShutdown {
     
     nm.stop();
     
-    // 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));
+    // 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;
+      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();
   }
 
-  public static void startContainer(NodeManager nm, FileContext localFS,
-      File scriptFileDir, File processStartFile) throws IOException,
-      YarnRemoteException {
+  public static void startContainer(NodeManager nm, ContainerId cId,
+      FileContext localFS, File scriptFileDir, File processStartFile) 
+          throws IOException, YarnRemoteException {
     File scriptFile =
-        createUnhaltingScriptFile(scriptFileDir, processStartFile);
+        createUnhaltingScriptFile(cId, scriptFileDir, processStartFile);
     
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     Container mockContainer = new ContainerPBImpl();
-    // Construct the Container-id
-    ContainerId cId = createContainerId();
     mockContainer.setId(cId);
 
     NodeId nodeId = BuilderUtils.newNodeId("localhost", 1234);
@@ -173,9 +187,7 @@ public class TestNodeManagerShutdown {
     localResources.put(destinationFile, localResource);
     containerLaunchContext.setLocalResources(localResources);
     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);
     Resource resource = BuilderUtils.newResource(1024, 1);
     mockContainer.setResource(resource);
@@ -239,18 +251,26 @@ public class TestNodeManagerShutdown {
    * Creates a script to run a container that will run forever unless
    * stopped by external means.
    */
-  private static File createUnhaltingScriptFile(File scriptFileDir,
-      File processStartFile) throws IOException {
+  private static File createUnhaltingScriptFile(ContainerId cId,
+      File scriptFileDir, File processStartFile) throws IOException {
     File scriptFile = new File(scriptFileDir, "scriptFile.sh");
-    BufferedWriter fileWriter = new BufferedWriter(new FileWriter(scriptFile));
-    fileWriter.write("#!/bin/bash\n\n");
-    fileWriter.write("echo \"Running testscript for delayed kill\"\n");
-    fileWriter.write("hello=\"Got SIGTERM\"\n");
-    fileWriter.write("umask 0\n");
-    fileWriter.write("trap \"echo $hello >> " + processStartFile + "\" SIGTERM\n");
-    fileWriter.write("echo \"Writing pid to start file\"\n");
-    fileWriter.write("echo $$ >> " + processStartFile + "\n");
-    fileWriter.write("while true; do\ndate >> /dev/null;\n done\n");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+    if (Shell.WINDOWS) {
+      fileWriter.println("@echo \"Running testscript for delayed kill\"");
+      fileWriter.println("@echo \"Writing pid to start file\"");
+      fileWriter.println("@echo " + cId + ">> " + processStartFile);
+      fileWriter.println("@pause");
+    } else {
+      fileWriter.write("#!/bin/bash\n\n");
+      fileWriter.write("echo \"Running testscript for delayed kill\"\n");
+      fileWriter.write("hello=\"Got SIGTERM\"\n");
+      fileWriter.write("umask 0\n");
+      fileWriter.write("trap \"echo $hello >> " + processStartFile +
+        "\" SIGTERM\n");
+      fileWriter.write("echo \"Writing pid to start file\"\n");
+      fileWriter.write("echo $$ >> " + processStartFile + "\n");
+      fileWriter.write("while true; do\ndate >> /dev/null;\n done\n");
+    }
 
     fileWriter.close();
     return scriptFile;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java

@@ -80,15 +80,15 @@ public abstract class BaseContainerManagerTest {
   public BaseContainerManagerTest() throws UnsupportedFileSystemException {
     localFS = FileContext.getLocalFSFileContext();
     localDir =
-        new File("target", this.getClass().getName() + "-localDir")
+        new File("target", this.getClass().getSimpleName() + "-localDir")
             .getAbsoluteFile();
     localLogDir =
-        new File("target", this.getClass().getName() + "-localLogDir")
+        new File("target", this.getClass().getSimpleName() + "-localLogDir")
             .getAbsoluteFile();
     remoteLogDir =
-      new File("target", this.getClass().getName() + "-remoteLogDir")
+      new File("target", this.getClass().getSimpleName() + "-remoteLogDir")
           .getAbsoluteFile();
-    tmpDir = new File("target", this.getClass().getName() + "-tmpDir");
+    tmpDir = new File("target", this.getClass().getSimpleName() + "-tmpDir");
   }
 
   protected static Log LOG = LogFactory

+ 45 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java

@@ -38,6 +38,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
@@ -58,6 +59,7 @@ import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 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.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
@@ -205,22 +207,29 @@ public class TestContainerManager extends BaseContainerManagerTest {
       InterruptedException, YarnRemoteException {
     containerManager.start();
 
-    File scriptFile = new File(tmpDir, "scriptFile.sh");
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
     PrintWriter fileWriter = new PrintWriter(scriptFile);
     File processStartFile =
         new File(tmpDir, "start_file.txt").getAbsoluteFile();
-    fileWriter.write("\numask 0"); // So that start file is readable by the test
-    fileWriter.write("\necho Hello World! > " + processStartFile);
-    fileWriter.write("\necho $$ >> " + processStartFile);
-    fileWriter.write("\nexec sleep 100");
+
+    // ////// Construct the Container-id
+    ContainerId cId = createContainerId();
+
+    if (Shell.WINDOWS) {
+      fileWriter.println("@echo Hello World!> " + 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 Hello World! > " + processStartFile);
+      fileWriter.write("\necho $$ >> " + processStartFile);
+      fileWriter.write("\nexec sleep 100");
+    }
     fileWriter.close();
 
     ContainerLaunchContext containerLaunchContext = 
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
-    // ////// Construct the Container-id
-    ContainerId cId = createContainerId();
-
     containerLaunchContext.setUser(user);
 
     URL resource_alpha =
@@ -239,14 +248,12 @@ public class TestContainerManager extends BaseContainerManagerTest {
     localResources.put(destinationFile, rsrc_alpha);
     containerLaunchContext.setLocalResources(localResources);
     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);
     Container mockContainer = mock(Container.class);
     when(mockContainer.getId()).thenReturn(cId);
     when(mockContainer.getResource()).thenReturn(
-        BuilderUtils.newResource(100 * 1024 * 1024, 1));
+        BuilderUtils.newResource(100, 1)); // MB
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(
         context.getNodeId().getHost() + ":12345");
@@ -278,12 +285,10 @@ public class TestContainerManager 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);
@@ -297,29 +302,40 @@ public class TestContainerManager 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));
   }
   
   private void testContainerLaunchAndExit(int exitCode) throws IOException,
       InterruptedException, YarnRemoteException {
 
-	  File scriptFile = new File(tmpDir, "scriptFile.sh");
+	  File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
 	  PrintWriter fileWriter = new PrintWriter(scriptFile);
 	  File processStartFile =
 			  new File(tmpDir, "start_file.txt").getAbsoluteFile();
-	  fileWriter.write("\numask 0"); // So that start file is readable by the test
-	  fileWriter.write("\necho Hello World! > " + processStartFile);
-	  fileWriter.write("\necho $$ >> " + processStartFile); 
 
-	  // Have script throw an exit code at the end
-	  if (exitCode != 0) {
-		  fileWriter.write("\nexit "+exitCode);
+	  // ////// Construct the Container-id
+	  ContainerId cId = createContainerId();
+
+	  if (Shell.WINDOWS) {
+	    fileWriter.println("@echo Hello World!> " + processStartFile);
+	    fileWriter.println("@echo " + cId + ">> " + processStartFile);
+	    if (exitCode != 0) {
+	      fileWriter.println("@exit " + exitCode);
+	    }
+	  } else {
+	    fileWriter.write("\numask 0"); // So that start file is readable by the test
+	    fileWriter.write("\necho Hello World! > " + processStartFile);
+	    fileWriter.write("\necho $$ >> " + processStartFile); 
+	    // Have script throw an exit code at the end
+	    if (exitCode != 0) {
+	      fileWriter.write("\nexit "+exitCode);
+	    }
 	  }
 	  
 	  fileWriter.close();
@@ -327,9 +343,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
 	  ContainerLaunchContext containerLaunchContext = 
 			  recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
-	  // ////// Construct the Container-id
-	  ContainerId cId = createContainerId();
-
 	  containerLaunchContext.setUser(user);
 
 	  URL resource_alpha =
@@ -348,14 +361,12 @@ public class TestContainerManager extends BaseContainerManagerTest {
 	  localResources.put(destinationFile, rsrc_alpha);
 	  containerLaunchContext.setLocalResources(localResources);
 	  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);
     Container mockContainer = mock(Container.class);
     when(mockContainer.getId()).thenReturn(cId);
     when(mockContainer.getResource()).thenReturn(
-        BuilderUtils.newResource(100 * 1024 * 1024, 1));
+        BuilderUtils.newResource(100, 1)); // MB
 
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(

+ 101 - 67
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -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();
   }
 
 }