You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2015/12/14 20:13:31 UTC

hadoop git commit: YARN-4309. Add container launch related debug information to container logs when a container fails. (Varun Vasudev via wangda)

Repository: hadoop
Updated Branches:
  refs/heads/trunk e53456981 -> dfcbbddb0


YARN-4309. Add container launch related debug information to container logs when a container fails. (Varun Vasudev via wangda)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dfcbbddb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dfcbbddb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dfcbbddb

Branch: refs/heads/trunk
Commit: dfcbbddb0963c89c0455d41223427165b9f9e537
Parents: e534569
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Dec 14 11:13:22 2015 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon Dec 14 11:13:22 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   5 +
 .../src/main/resources/yarn-default.xml         |  12 +++
 .../server/nodemanager/ContainerExecutor.java   |  24 ++++-
 .../nodemanager/DockerContainerExecutor.java    |  11 +-
 .../launcher/ContainerLaunch.java               |  74 +++++++++++++-
 .../launcher/TestContainerLaunch.java           | 102 +++++++++++++++++--
 7 files changed, 217 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfcbbddb/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7f12520..82c4314 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -613,6 +613,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3946. Update exact reason as to why a submitted app is in ACCEPTED state to 
     app's diagnostic message. (Naganarasimha G R via wangda)
 
+    YARN-4309. Add container launch related debug information to container logs
+    when a container fails. (Varun Vasudev via wangda)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfcbbddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 2d9ca72..54ad422 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -92,6 +92,11 @@ public class YarnConfiguration extends Configuration {
   /** Delay before deleting resource to ease debugging of NM issues */
   public static final String DEBUG_NM_DELETE_DELAY_SEC =
     YarnConfiguration.NM_PREFIX + "delete.debug-delay-sec";
+
+  public static final String NM_LOG_CONTAINER_DEBUG_INFO =
+      YarnConfiguration.NM_PREFIX + "log-container-debug-info.enabled";
+
+  public static final boolean DEFAULT_NM_LOG_CONTAINER_DEBUG_INFO = false;
   
   ////////////////////////////////
   // IPC Configs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfcbbddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 5b71b1f..13a7b1b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1128,6 +1128,18 @@
   </property>
 
   <property>
+    <description>Generate additional logs about container launches.
+    Currently, this creates a copy of the launch script and lists the
+    directory contents of the container work dir. When listing directory
+    contents, we follow symlinks to a max-depth of 5(including symlinks
+    which point to outside the container work dir) which may lead to a
+    slowness in launching containers.
+    </description>
+    <name>yarn.nodemanager.log-container-debug-info.enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
     <description>Amount of physical memory, in MB, that can be allocated 
     for containers. If set to -1 and
     yarn.nodemanager.resource.detect-hardware-capabilities is true, it is

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfcbbddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
index 6d75a1c..8c74bf5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
@@ -34,6 +34,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -65,6 +66,8 @@ public abstract class ContainerExecutor implements Configurable {
   final public static FsPermission TASK_LAUNCH_SCRIPT_PERMISSION =
     FsPermission.createImmutable((short) 0700);
 
+  public static final String DIRECTORY_CONTENTS = "directory.info";
+
   private Configuration conf;
 
   private ConcurrentMap<ContainerId, Path> pidFiles =
@@ -241,11 +244,22 @@ public abstract class ContainerExecutor implements Configurable {
    * @param resources The resources which have been localized for this container
    * Symlinks will be created to these localized resources
    * @param command The command that will be run.
+   * @param logDir The log dir to copy debugging information to
    * @throws IOException if any errors happened writing to the OutputStream,
    * while creating symlinks
    */
   public void writeLaunchEnv(OutputStream out, Map<String, String> environment,
-    Map<Path, List<String>> resources, List<String> command) throws IOException{
+      Map<Path, List<String>> resources, List<String> command, Path logDir)
+      throws IOException {
+    this.writeLaunchEnv(out, environment, resources, command, logDir,
+        ContainerLaunch.CONTAINER_SCRIPT);
+  }
+
+  @VisibleForTesting
+  public void writeLaunchEnv(OutputStream out,
+      Map<String, String> environment, Map<Path, List<String>> resources,
+      List<String> command, Path logDir, String outFilename)
+      throws IOException {
     ContainerLaunch.ShellScriptBuilder sb =
       ContainerLaunch.ShellScriptBuilder.create();
     Set<String> whitelist = new HashSet<String>();
@@ -272,6 +286,14 @@ public abstract class ContainerExecutor implements Configurable {
       }
     }
 
+    // dump debugging information if configured
+    if (getConf() != null && getConf().getBoolean(
+        YarnConfiguration.NM_LOG_CONTAINER_DEBUG_INFO,
+        YarnConfiguration.DEFAULT_NM_LOG_CONTAINER_DEBUG_INFO)) {
+      sb.copyDebugInformation(new Path(outFilename), new Path(logDir, outFilename));
+      sb.listDebugInformation(new Path(logDir, DIRECTORY_CONTENTS));
+    }
+
     sb.command(command);
 
     PrintStream pout = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfcbbddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
index 96f8b51..b089947 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
@@ -329,7 +329,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
    * the docker image and write them out to an OutputStream.
    */
   public void writeLaunchEnv(OutputStream out, Map<String, String> environment,
-    Map<Path, List<String>> resources, List<String> command)
+    Map<Path, List<String>> resources, List<String> command, Path logDir)
     throws IOException {
     ContainerLaunch.ShellScriptBuilder sb =
       ContainerLaunch.ShellScriptBuilder.create();
@@ -358,6 +358,15 @@ public class DockerContainerExecutor extends ContainerExecutor {
       }
     }
 
+    // dump debugging information if configured
+    if (getConf() != null && getConf().getBoolean(
+        YarnConfiguration.NM_LOG_CONTAINER_DEBUG_INFO,
+        YarnConfiguration.DEFAULT_NM_LOG_CONTAINER_DEBUG_INFO)) {
+      sb.copyDebugInformation(new Path(ContainerLaunch.CONTAINER_SCRIPT),
+          new Path(logDir, ContainerLaunch.CONTAINER_SCRIPT));
+      sb.listDebugInformation(new Path(logDir, DIRECTORY_CONTENTS));
+    }
+
     sb.command(command);
 
     PrintStream pout = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfcbbddb/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
----------------------------------------------------------------------
diff --git a/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 b/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
index fc8615b..b7a97d6 100644
--- a/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
+++ b/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
@@ -272,7 +272,8 @@ public class ContainerLaunch implements Callable<Integer> {
 
         // Write out the environment
         exec.writeLaunchEnv(containerScriptOutStream, environment,
-          localResources, launchContext.getCommands());
+          localResources, launchContext.getCommands(),
+            new Path(containerLogDirs.get(0)));
 
         // /////////// End of writing out container-script
 
@@ -641,6 +642,28 @@ public class ContainerLaunch implements Callable<Integer> {
       link(src, dst);
     }
 
+    /**
+     * Method to copy files that are useful for debugging container failures.
+     * This method will be called by ContainerExecutor when setting up the
+     * container launch script. The method should take care to make sure files
+     * are read-able by the yarn user if the files are to undergo
+     * log-aggregation.
+     * @param src path to the source file
+     * @param dst path to the destination file - should be absolute
+     * @throws IOException
+     */
+    public abstract void copyDebugInformation(Path src, Path dst)
+        throws IOException;
+
+    /**
+     * Method to dump debug information to the a target file. This method will
+     * be called by ContainerExecutor when setting up the container launch
+     * script.
+     * @param output the file to which debug information is to be written
+     * @throws IOException
+     */
+    public abstract void listDebugInformation(Path output) throws IOException;
+
     @Override
     public String toString() {
       return sb.toString();
@@ -704,6 +727,36 @@ public class ContainerLaunch implements Callable<Integer> {
       line("mkdir -p ", path.toString());
       errorCheck();
     }
+
+    @Override
+    public void copyDebugInformation(Path src, Path dest) throws IOException {
+      line("# Creating copy of launch script");
+      line("cp \"", src.toUri().getPath(), "\" \"", dest.toUri().getPath(),
+          "\"");
+      // set permissions to 640 because we need to be able to run
+      // log aggregation in secure mode as well
+      if(dest.isAbsolute()) {
+        line("chmod 640 \"", dest.toUri().getPath(), "\"");
+      }
+    }
+
+    @Override
+    public void listDebugInformation(Path output) throws  IOException {
+      line("# Determining directory contents");
+      line("echo \"ls -l:\" 1>\"", output.toString(), "\"");
+      line("ls -l 1>>\"", output.toString(), "\"");
+
+      // don't run error check because if there are loops
+      // find will exit with an error causing container launch to fail
+      // find will follow symlinks outside the work dir if such sylimks exist
+      // (like public/app local resources)
+      line("echo \"find -L . -maxdepth 5 -ls:\" 1>>\"", output.toString(),
+          "\"");
+      line("find -L . -maxdepth 5 -ls 1>>\"", output.toString(), "\"");
+      line("echo \"broken symlinks(find -L . -maxdepth 5 -type l -ls):\" 1>>\"",
+          output.toString(), "\"");
+      line("find -L . -maxdepth 5 -type l -ls 1>>\"", output.toString(), "\"");
+    }
   }
 
   private static final class WindowsShellScriptBuilder
@@ -757,6 +810,25 @@ public class ContainerLaunch implements Callable<Integer> {
           path.toString(), path.toString()));
       errorCheck();
     }
+
+    @Override
+    public void copyDebugInformation(Path src, Path dest)
+        throws IOException {
+      // no need to worry about permissions - in secure mode
+      // WindowsSecureContainerExecutor will set permissions
+      // to allow NM to read the file
+      line("rem Creating copy of launch script");
+      lineWithLenCheck(String.format("copy \"%s\" \"%s\"", src.toString(),
+          dest.toString()));
+    }
+
+    @Override
+    public void listDebugInformation(Path output) throws IOException {
+      line("rem Determining directory contents");
+      lineWithLenCheck(
+          String.format("@echo \"dir:\" > \"%s\"", output.toString()));
+      lineWithLenCheck(String.format("dir >> \"%s\"", output.toString()));
+    }
   }
 
   private static void putEnvIfNotNull(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfcbbddb/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 0abae2b..502ecde 100644
--- a/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
+++ b/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
@@ -100,6 +100,7 @@ import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
+import org.hamcrest.CoreMatchers;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
@@ -166,7 +167,9 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
         commands.add("/bin/sh ./\\\"" + badSymlink + "\\\"");
       }
 
-      new DefaultContainerExecutor().writeLaunchEnv(fos, env, resources, commands);
+      new DefaultContainerExecutor()
+          .writeLaunchEnv(fos, env, resources, commands,
+              new Path(localLogDir.getAbsolutePath()), tempFile.getName());
       fos.flush();
       fos.close();
       FileUtil.setExecutable(tempFile, true);
@@ -233,7 +236,9 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       } else {
         commands.add("/bin/sh ./\\\"" + symLink + "\\\"");
       }
-      new DefaultContainerExecutor().writeLaunchEnv(fos, env, resources, commands);
+      new DefaultContainerExecutor()
+          .writeLaunchEnv(fos, env, resources, commands,
+              new Path(localLogDir.getAbsolutePath()));
       fos.flush();
       fos.close();
       FileUtil.setExecutable(tempFile, true);
@@ -286,7 +291,9 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
           "\"workflowName\":\"\n\ninsert table " +
           "\npartition (cd_education_status)\nselect cd_demo_sk, cd_gender, " );
       List<String> commands = new ArrayList<String>();
-      new DefaultContainerExecutor().writeLaunchEnv(fos, env, resources, commands);
+      new DefaultContainerExecutor()
+          .writeLaunchEnv(fos, env, resources, commands,
+              new Path(localLogDir.getAbsolutePath()));
       fos.flush();
       fos.close();
 
@@ -364,7 +371,8 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       List<String> commands = new ArrayList<String>();
       commands.add(command);
       ContainerExecutor exec = new DefaultContainerExecutor();
-      exec.writeLaunchEnv(fos, env, resources, commands);
+      exec.writeLaunchEnv(fos, env, resources, commands,
+          new Path(localLogDir.getAbsolutePath()));
       fos.flush();
       fos.close();
 
@@ -994,7 +1002,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
               "X", Shell.WINDOWS_MAX_SHELL_LENGTH -callCmd.length() + 1)));
       fail("longCommand was expected to throw");
     } catch(IOException e) {
-      assertThat(e.getMessage(), containsString(expectedMessage));
+      assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage));
     }
 
     // Composite tests, from parts: less, exact and +
@@ -1016,7 +1024,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
           org.apache.commons.lang.StringUtils.repeat("X", 2048 - callCmd.length())));
       fail("long commands was expected to throw");
     } catch(IOException e) {
-      assertThat(e.getMessage(), containsString(expectedMessage));
+      assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage));
     }
   }
   
@@ -1039,7 +1047,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
           "A", Shell.WINDOWS_MAX_SHELL_LENGTH - ("@set somekey=").length()) + 1);
       fail("long env was expected to throw");
     } catch(IOException e) {
-      assertThat(e.getMessage(), containsString(expectedMessage));
+      assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage));
     }
   }
     
@@ -1057,14 +1065,14 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
 
     // test mkdir
     builder.mkdir(new Path(org.apache.commons.lang.StringUtils.repeat("A", 1024)));
-    builder.mkdir(new Path(org.apache.commons.lang.StringUtils.repeat(
-        "E", (Shell.WINDOWS_MAX_SHELL_LENGTH - mkDirCmd.length())/2)));
+    builder.mkdir(new Path(org.apache.commons.lang.StringUtils.repeat("E",
+        (Shell.WINDOWS_MAX_SHELL_LENGTH - mkDirCmd.length()) / 2)));
     try {
       builder.mkdir(new Path(org.apache.commons.lang.StringUtils.repeat(
           "X", (Shell.WINDOWS_MAX_SHELL_LENGTH - mkDirCmd.length())/2 +1)));
       fail("long mkdir was expected to throw");
     } catch(IOException e) {
-      assertThat(e.getMessage(), containsString(expectedMessage));
+      assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage));
     }    
   }
 
@@ -1095,7 +1103,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
               "Y", (Shell.WINDOWS_MAX_SHELL_LENGTH - linkCmd.length())/2) + 1));
       fail("long link was expected to throw");
     } catch(IOException e) {
-      assertThat(e.getMessage(), containsString(expectedMessage));
+      assertThat(e.getMessage(), CoreMatchers.containsString(expectedMessage));
     }
   }
 
@@ -1208,4 +1216,76 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     Assert.assertEquals(ExitCode.FORCE_KILLED.getExitCode(),
         containerStatus.getExitStatus());
   }
+
+  @Test
+  public void testDebuggingInformation() throws IOException {
+
+    File shellFile = null;
+    File tempFile = null;
+    Configuration conf = new YarnConfiguration();
+    try {
+      shellFile = Shell.appendScriptExtension(tmpDir, "hello");
+      tempFile = Shell.appendScriptExtension(tmpDir, "temp");
+      String testCommand = Shell.WINDOWS ? "@echo \"hello\"" :
+          "echo \"hello\"";
+      PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));
+      FileUtil.setExecutable(shellFile, true);
+      writer.println(testCommand);
+      writer.close();
+
+      Map<Path, List<String>> resources = new HashMap<Path, List<String>>();
+      Map<String, String> env = new HashMap<String, String>();
+      List<String> commands = new ArrayList<String>();
+      if (Shell.WINDOWS) {
+        commands.add("cmd");
+        commands.add("/c");
+        commands.add("\"" + shellFile.getAbsolutePath() + "\"");
+      } else {
+        commands.add("/bin/sh \\\"" + shellFile.getAbsolutePath() + "\\\"");
+      }
+
+      boolean[] debugLogsExistArray = { false, true };
+      for (boolean debugLogsExist : debugLogsExistArray) {
+
+        conf.setBoolean(YarnConfiguration.NM_LOG_CONTAINER_DEBUG_INFO,
+          debugLogsExist);
+        FileOutputStream fos = new FileOutputStream(tempFile);
+        ContainerExecutor exec = new DefaultContainerExecutor();
+        exec.setConf(conf);
+        exec.writeLaunchEnv(fos, env, resources, commands,
+          new Path(localLogDir.getAbsolutePath()), tempFile.getName());
+        fos.flush();
+        fos.close();
+        FileUtil.setExecutable(tempFile, true);
+
+        Shell.ShellCommandExecutor shexc = new Shell.ShellCommandExecutor(
+          new String[] { tempFile.getAbsolutePath() }, tmpDir);
+
+        shexc.execute();
+        assertEquals(shexc.getExitCode(), 0);
+        File directorInfo =
+          new File(localLogDir, ContainerExecutor.DIRECTORY_CONTENTS);
+        File scriptCopy = new File(localLogDir, tempFile.getName());
+
+        Assert.assertEquals("Directory info file missing", debugLogsExist,
+          directorInfo.exists());
+        Assert.assertEquals("Copy of launch script missing", debugLogsExist,
+          scriptCopy.exists());
+        if (debugLogsExist) {
+          Assert.assertTrue("Directory info file size is 0",
+            directorInfo.length() > 0);
+          Assert.assertTrue("Size of copy of launch script is 0",
+            scriptCopy.length() > 0);
+        }
+      }
+    } finally {
+      // cleanup
+      if (shellFile != null && shellFile.exists()) {
+        shellFile.delete();
+      }
+      if (tempFile != null && tempFile.exists()) {
+        tempFile.delete();
+      }
+    }
+  }
 }