Ver código fonte

YARN-6550. Capture launch_container.sh logs to a separate log file. (Suma Shivaprasad via wangda)

Change-Id: I0ee0b1bb459437432a22cf68861a6354f0decabb
(cherry picked from commit febeead5f95c6fc245ea3735f5b538d4bb4dc8a4)
Wangda Tan 7 anos atrás
pai
commit
cdaab892d8

+ 15 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java

@@ -64,6 +64,9 @@ import org.apache.hadoop.yarn.server.nodemanager.util.ProcessIdFileReader;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.CONTAINER_PRE_LAUNCH_STDERR;
+import static org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.CONTAINER_PRE_LAUNCH_STDOUT;
+
 /**
  * This class is abstraction of the mechanism used to launch a container on the
  * underlying OS.  All executor implementations must extend ContainerExecutor.
@@ -330,6 +333,14 @@ public abstract class ContainerExecutor implements Configurable {
       String user, String outFilename) throws IOException {
     ContainerLaunch.ShellScriptBuilder sb =
         ContainerLaunch.ShellScriptBuilder.create();
+
+    // Add "set -o pipefail -e" to validate launch_container script.
+    sb.setExitOnFailure();
+
+    //Redirect stdout and stderr for launch_container script
+    sb.stdout(logDir, CONTAINER_PRE_LAUNCH_STDOUT);
+    sb.stderr(logDir, CONTAINER_PRE_LAUNCH_STDERR);
+
     Set<String> whitelist = new HashSet<>();
 
     String[] nmWhiteList = conf.get(YarnConfiguration.NM_ENV_WHITELIST,
@@ -338,10 +349,8 @@ public abstract class ContainerExecutor implements Configurable {
       whitelist.add(param);
     }
 
-    // Add "set -o pipefail -e" to validate launch_container script.
-    sb.setExitOnFailure();
-
     if (environment != null) {
+      sb.echo("Setting up env variables");
       for (Map.Entry<String, String> env : environment.entrySet()) {
         if (!whitelist.contains(env.getKey())) {
           sb.env(env.getKey(), env.getValue());
@@ -352,6 +361,7 @@ public abstract class ContainerExecutor implements Configurable {
     }
 
     if (resources != null) {
+      sb.echo("Setting up job resources");
       for (Map.Entry<Path, List<String>> resourceEntry :
           resources.entrySet()) {
         for (String linkName : resourceEntry.getValue()) {
@@ -373,15 +383,15 @@ public abstract class ContainerExecutor implements Configurable {
     if (getConf() != null &&
         getConf().getBoolean(YarnConfiguration.NM_LOG_CONTAINER_DEBUG_INFO,
         YarnConfiguration.DEFAULT_NM_LOG_CONTAINER_DEBUG_INFO)) {
+      sb.echo("Copying debugging information");
       sb.copyDebugInformation(new Path(outFilename),
           new Path(logDir, outFilename));
       sb.listDebugInformation(new Path(logDir, DIRECTORY_CONTENTS));
     }
-
+    sb.echo("Launching container");
     sb.command(command);
 
     PrintStream pout = null;
-
     try {
       pout = new PrintStream(out, false, "UTF-8");
       sb.write(pout);

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java

@@ -320,8 +320,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
           builder.append("Exception message: ");
           builder.append(e.getMessage()).append("\n");
         }
-        builder.append("Stack trace: ");
-        builder.append(StringUtils.stringifyException(e)).append("\n");
+
         if (!shExec.getOutput().isEmpty()) {
           builder.append("Shell output: ");
           builder.append(shExec.getOutput()).append("\n");

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -532,8 +532,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
         if (!Optional.fromNullable(e.getErrorOutput()).or("").isEmpty()) {
           builder.append("Exception message: " + e.getErrorOutput() + "\n");
         }
-        builder.append("Stack trace: "
-            + StringUtils.stringifyException(e) + "\n");
+        //Skip stack trace
         String output = e.getOutput();
         if (output != null && !e.getOutput().isEmpty()) {
           builder.append("Shell output: " + output + "\n");

+ 160 - 29
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

@@ -87,6 +87,8 @@ import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 public class ContainerLaunch implements Callable<Integer> {
@@ -94,8 +96,13 @@ public class ContainerLaunch implements Callable<Integer> {
   private static final Logger LOG =
        LoggerFactory.getLogger(ContainerLaunch.class);
 
+  private static final String CONTAINER_PRE_LAUNCH_PREFIX = "prelaunch";
+  public static final String CONTAINER_PRE_LAUNCH_STDOUT = CONTAINER_PRE_LAUNCH_PREFIX + ".out";
+  public static final String CONTAINER_PRE_LAUNCH_STDERR = CONTAINER_PRE_LAUNCH_PREFIX + ".err";
+
   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";
@@ -147,7 +154,7 @@ public class ContainerLaunch implements Callable<Integer> {
       Path containerLogDir) {
     var = var.replace(ApplicationConstants.LOG_DIR_EXPANSION_VAR,
       containerLogDir.toString());
-    var =  var.replace(ApplicationConstants.CLASS_PATH_SEPARATOR,
+    var = var.replace(ApplicationConstants.CLASS_PATH_SEPARATOR,
       File.pathSeparator);
 
     // replace parameter expansion marker. e.g. {{VAR}} on Windows is replaced
@@ -371,7 +378,7 @@ public class ContainerLaunch implements Callable<Integer> {
     String relativeContainerLogDir = ContainerLaunch
         .getRelativeContainerLogDir(appIdStr, containerIdStr);
 
-    for(String logDir : logDirs) {
+    for (String logDir : logDirs) {
       containerLogDirs.add(logDir + Path.SEPARATOR + relativeContainerLogDir);
     }
 
@@ -516,6 +523,7 @@ public class ContainerLaunch implements Callable<Integer> {
    * Tries to tail and fetch TAIL_SIZE_IN_BYTES of data from the error log.
    * ErrorLog filename is not fixed and depends upon app, hence file name
    * pattern is used.
+   *
    * @param containerID
    * @param ret
    * @param containerLogDir
@@ -524,20 +532,46 @@ public class ContainerLaunch implements Callable<Integer> {
   @SuppressWarnings("unchecked")
   protected void handleContainerExitWithFailure(ContainerId containerID,
       int ret, Path containerLogDir, StringBuilder diagnosticInfo) {
-    LOG.warn(diagnosticInfo.toString());
+    LOG.warn("Container launch failed : " + diagnosticInfo.toString());
+
+    FileSystem fileSystem = null;
+    long tailSizeInBytes =
+        conf.getLong(YarnConfiguration.NM_CONTAINER_STDERR_BYTES,
+            YarnConfiguration.DEFAULT_NM_CONTAINER_STDERR_BYTES);
+
+    // Append container prelaunch stderr to diagnostics
+    try {
+      fileSystem = FileSystem.getLocal(conf).getRaw();
+      FileStatus preLaunchErrorFileStatus = fileSystem
+          .getFileStatus(new Path(containerLogDir, ContainerLaunch.CONTAINER_PRE_LAUNCH_STDERR));
+
+      Path errorFile = preLaunchErrorFileStatus.getPath();
+      long fileSize = preLaunchErrorFileStatus.getLen();
+
+      diagnosticInfo.append("Error file: ")
+          .append(ContainerLaunch.CONTAINER_PRE_LAUNCH_STDERR).append(".\n");
+      ;
 
+      byte[] tailBuffer = tailFile(errorFile, fileSize, tailSizeInBytes);
+      diagnosticInfo.append("Last ").append(tailSizeInBytes)
+          .append(" bytes of ").append(errorFile.getName()).append(" :\n")
+          .append(new String(tailBuffer, StandardCharsets.UTF_8));
+    } catch (IOException e) {
+      LOG.error("Failed to get tail of the container's prelaunch error log file", e);
+    }
+
+    // Append container stderr to diagnostics
     String errorFileNamePattern =
         conf.get(YarnConfiguration.NM_CONTAINER_STDERR_PATTERN,
             YarnConfiguration.DEFAULT_NM_CONTAINER_STDERR_PATTERN);
-    FSDataInputStream errorFileIS = null;
+
     try {
-      FileSystem fileSystem = FileSystem.getLocal(conf).getRaw();
+      if (fileSystem == null) {
+        fileSystem = FileSystem.getLocal(conf).getRaw();
+      }
       FileStatus[] errorFileStatuses = fileSystem
           .globStatus(new Path(containerLogDir, errorFileNamePattern));
       if (errorFileStatuses != null && errorFileStatuses.length != 0) {
-        long tailSizeInBytes =
-            conf.getLong(YarnConfiguration.NM_CONTAINER_STDERR_BYTES,
-                YarnConfiguration.DEFAULT_NM_CONTAINER_STDERR_BYTES);
         Path errorFile = errorFileStatuses[0].getPath();
         long fileSize = errorFileStatuses[0].getLen();
 
@@ -560,32 +594,40 @@ public class ContainerLaunch implements Callable<Integer> {
               .append(StringUtils.join(", ", errorFileNames)).append(".\n");
         }
 
-        long startPosition =
-            (fileSize < tailSizeInBytes) ? 0 : fileSize - tailSizeInBytes;
-        int bufferSize =
-            (int) ((fileSize < tailSizeInBytes) ? fileSize : tailSizeInBytes);
-        byte[] tailBuffer = new byte[bufferSize];
-        errorFileIS = fileSystem.open(errorFile);
-        errorFileIS.readFully(startPosition, tailBuffer);
-
+        byte[] tailBuffer = tailFile(errorFile, fileSize, tailSizeInBytes);
         String tailBufferMsg = new String(tailBuffer, StandardCharsets.UTF_8);
         diagnosticInfo.append("Last ").append(tailSizeInBytes)
             .append(" bytes of ").append(errorFile.getName()).append(" :\n")
             .append(tailBufferMsg).append("\n")
             .append(analysesErrorMsgOfContainerExitWithFailure(tailBufferMsg));
+
       }
     } catch (IOException e) {
       LOG.error("Failed to get tail of the container's error log file", e);
-    } finally {
-      IOUtils.cleanupWithLogger(LOG, errorFileIS);
     }
-
     this.dispatcher.getEventHandler()
         .handle(new ContainerExitEvent(containerID,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, ret,
             diagnosticInfo.toString()));
   }
 
+  private byte[] tailFile(Path filePath, long fileSize, long tailSizeInBytes) throws IOException {
+    FSDataInputStream errorFileIS = null;
+    FileSystem fileSystem = FileSystem.getLocal(conf).getRaw();
+    try {
+      long startPosition =
+          (fileSize < tailSizeInBytes) ? 0 : fileSize - tailSizeInBytes;
+      int bufferSize =
+          (int) ((fileSize < tailSizeInBytes) ? fileSize : tailSizeInBytes);
+      byte[] tailBuffer = new byte[bufferSize];
+      errorFileIS = fileSystem.open(filePath);
+      errorFileIS.readFully(startPosition, tailBuffer);
+      return tailBuffer;
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, errorFileIS);
+    }
+  }
+
   private String analysesErrorMsgOfContainerExitWithFailure(String errorMsg) {
     StringBuilder analysis = new StringBuilder();
     if (errorMsg.indexOf("Error: Could not find or load main class"
@@ -982,8 +1024,48 @@ public class ContainerLaunch implements Callable<Integer> {
 
     public abstract void whitelistedEnv(String key, String value) throws IOException;
 
+    protected static final String ENV_PRELAUNCH_STDOUT = "PRELAUNCH_OUT";
+    protected static final String ENV_PRELAUNCH_STDERR = "PRELAUNCH_ERR";
+
+    private boolean redirectStdOut = false;
+    private boolean redirectStdErr = false;
+
+    /**
+     * Set stdout for the shell script
+     * @param stdoutDir stdout must be an absolute path
+     * @param stdOutFile stdout file name
+     * @throws IOException thrown when stdout path is not absolute
+     */
+    public final void stdout(Path stdoutDir, String stdOutFile) throws IOException {
+      if (!stdoutDir.isAbsolute()) {
+        throw new IOException("Stdout path must be absolute");
+      }
+      redirectStdOut = true;
+      setStdOut(new Path(stdoutDir, stdOutFile));
+    }
+
+    /**
+     * Set stderr for the shell script
+     * @param stderrDir stderr must be an absolute path
+     * @param stdErrFile stderr file name
+     * @throws IOException thrown when stderr path is not absolute
+     */
+    public final void stderr(Path stderrDir, String stdErrFile) throws IOException {
+      if (!stderrDir.isAbsolute()) {
+        throw new IOException("Stdout path must be absolute");
+      }
+      redirectStdErr = true;
+      setStdErr(new Path(stderrDir, stdErrFile));
+    }
+
+    protected abstract void setStdOut(Path stdout) throws IOException;
+
+    protected abstract void setStdErr(Path stdout) throws IOException;
+
     public abstract void env(String key, String value) throws IOException;
 
+    public abstract void echo(String echoStr) throws IOException;
+
     public final void symlink(Path src, Path dst) throws IOException {
       if (!src.isAbsolute()) {
         throw new IOException("Source must be absolute");
@@ -1028,13 +1110,21 @@ public class ContainerLaunch implements Callable<Integer> {
       out.append(sb);
     }
 
-    protected final void line(String... command) {
+    protected final void buildCommand(String... command) {
       for (String s : command) {
         sb.append(s);
       }
+    }
+
+    protected final void linebreak(String... command) {
       sb.append(LINE_SEPARATOR);
     }
 
+    protected final void line(String... command) {
+      buildCommand(command);
+      linebreak();
+    }
+
     public void setExitOnFailure() {
       // Dummy implementation
     }
@@ -1042,19 +1132,27 @@ public class ContainerLaunch implements Callable<Integer> {
     protected abstract void link(Path src, Path dst) throws IOException;
 
     protected abstract void mkdir(Path path) throws IOException;
+
+    boolean doRedirectStdOut() {
+      return redirectStdOut;
+    }
+
+    boolean doRedirectStdErr() {
+      return redirectStdErr;
+    }
+
   }
 
   private static final class UnixShellScriptBuilder extends ShellScriptBuilder {
-
     private void errorCheck() {
       line("hadoop_shell_errorcode=$?");
-      line("if [ $hadoop_shell_errorcode -ne 0 ]");
+      line("if [[ \"$hadoop_shell_errorcode\" -ne 0 ]]");
       line("then");
       line("  exit $hadoop_shell_errorcode");
       line("fi");
     }
 
-    public UnixShellScriptBuilder(){
+    public UnixShellScriptBuilder() {
       line("#!/bin/bash");
       line();
     }
@@ -1062,29 +1160,47 @@ public class ContainerLaunch implements Callable<Integer> {
     @Override
     public void command(List<String> command) {
       line("exec /bin/bash -c \"", StringUtils.join(" ", command), "\"");
-      errorCheck();
     }
 
     @Override
-    public void whitelistedEnv(String key, String value) {
+    public void whitelistedEnv(String key, String value) throws IOException {
       line("export ", key, "=${", key, ":-", "\"", value, "\"}");
     }
 
     @Override
-    public void env(String key, String value) {
+    public void setStdOut(final Path stdout) throws IOException {
+      line("export ", ENV_PRELAUNCH_STDOUT, "=\"", stdout.toString(), "\"");
+      // tee is needed for DefaultContainerExecutor error propagation to stdout
+      // Close stdout of subprocess to prevent it from writing to the stdout file
+      line("exec >\"${" + ENV_PRELAUNCH_STDOUT + "}\"");
+    }
+
+    @Override
+    public void setStdErr(final Path stderr) throws IOException {
+      line("export ", ENV_PRELAUNCH_STDERR, "=\"", stderr.toString(), "\"");
+      // tee is needed for DefaultContainerExecutor error propagation to stderr
+      // Close stdout of subprocess to prevent it from writing to the stdout file
+      line("exec 2>\"${" + ENV_PRELAUNCH_STDERR + "}\"");
+    }
+
+    @Override
+    public void env(String key, String value) throws IOException {
       line("export ", key, "=\"", value, "\"");
     }
 
+    @Override
+    public void echo(final String echoStr) throws IOException {
+      line("echo \"" + echoStr + "\"");
+    }
+
     @Override
     protected void link(Path src, Path dst) throws IOException {
       line("ln -sf \"", src.toUri().getPath(), "\" \"", dst.toString(), "\"");
-      errorCheck();
     }
 
     @Override
-    protected void mkdir(Path path) {
+    protected void mkdir(Path path) throws IOException {
       line("mkdir -p ", path.toString());
-      errorCheck();
     }
 
     @Override
@@ -1152,12 +1268,27 @@ public class ContainerLaunch implements Callable<Integer> {
       errorCheck();
     }
 
+    //Dummy implementation
+    @Override
+    protected void setStdOut(final Path stdout) throws IOException {
+    }
+
+    //Dummy implementation
+    @Override
+    protected void setStdErr(final Path stderr) throws IOException {
+    }
+
     @Override
     public void env(String key, String value) throws IOException {
       lineWithLenCheck("@set ", key, "=", value);
       errorCheck();
     }
 
+    @Override
+    public void echo(final String echoStr) throws IOException {
+      lineWithLenCheck("@echo \"", echoStr, "\"");
+    }
+
     @Override
     protected void link(Path src, Path dst) throws IOException {
       File srcFile = new File(src.toUri().getPath());

+ 113 - 2
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

@@ -32,6 +32,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
@@ -192,7 +193,11 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
 
       shexc.execute();
       assertEquals(shexc.getExitCode(), 0);
-      assert(shexc.getOutput().contains("hello"));
+      //Capture output from prelaunch.out
+
+      List<String> output = Files.readAllLines(Paths.get(localLogDir.getAbsolutePath(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDOUT),
+          Charset.forName("UTF-8"));
+      assert(output.contains("hello"));
 
       symLinkFile = new File(tmpDir, badSymlink);
     }
@@ -358,7 +363,10 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
         shexc.execute();
         Assert.fail("Should catch exception");
       } catch(ExitCodeException e){
-        diagnostics = e.getMessage();
+        //Capture diagnostics from prelaunch.stderr
+        List<String> error = Files.readAllLines(Paths.get(localLogDir.getAbsolutePath(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDERR),
+            Charset.forName("UTF-8"));
+        diagnostics = StringUtils.join("\n", error);
       }
       Assert.assertTrue(diagnostics.contains(Shell.WINDOWS ?
           "is not recognized as an internal or external command" :
@@ -1545,6 +1553,109 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
 
   }
 
+  /**
+   * Test that script exists with non-zero exit code when command fails.
+   * @throws IOException
+   */
+  @Test
+  public void testShellScriptBuilderStdOutandErrRedirection() throws IOException {
+    ShellScriptBuilder builder = ShellScriptBuilder.create();
+
+    Path logDir = new Path(localLogDir.getAbsolutePath());
+    File stdout = new File(logDir.toString(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDOUT);
+    File stderr = new File(logDir.toString(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDERR);
+
+    builder.stdout(logDir, ContainerLaunch.CONTAINER_PRE_LAUNCH_STDOUT);
+    builder.stderr(logDir, ContainerLaunch.CONTAINER_PRE_LAUNCH_STDERR);
+
+    //should redirect to specified stdout path
+    String TEST_STDOUT_ECHO = "Test stdout redirection";
+    builder.echo(TEST_STDOUT_ECHO);
+    //should fail and redirect to stderr
+    builder.mkdir(new Path("/invalidSrcDir"));
+
+    builder.command(Arrays.asList(new String[] {"unknownCommand"}));
+
+    File shellFile = Shell.appendScriptExtension(tmpDir, "testShellScriptBuilderStdOutandErrRedirection");
+    PrintStream writer = new PrintStream(new FileOutputStream(shellFile));
+    builder.write(writer);
+    writer.close();
+    try {
+      FileUtil.setExecutable(shellFile, true);
+
+      Shell.ShellCommandExecutor shexc = new Shell.ShellCommandExecutor(
+          new String[]{shellFile.getAbsolutePath()}, tmpDir);
+      try {
+        shexc.execute();
+        fail("builder shell command was expected to throw");
+      }
+      catch(IOException e) {
+        // expected
+        System.out.println("Received an expected exception: " + e.getMessage());
+
+        Assert.assertEquals(true, stdout.exists());
+        BufferedReader stdoutReader = new BufferedReader(new FileReader(stdout));
+        // Get the pid of the process
+        String line = stdoutReader.readLine().trim();
+        Assert.assertEquals(TEST_STDOUT_ECHO, line);
+        // No more lines
+        Assert.assertEquals(null, stdoutReader.readLine());
+        stdoutReader.close();
+
+        Assert.assertEquals(true, stderr.exists());
+        Assert.assertTrue(stderr.length() > 0);
+      }
+    }
+    finally {
+      FileUtil.fullyDelete(shellFile);
+      FileUtil.fullyDelete(stdout);
+      FileUtil.fullyDelete(stderr);
+    }
+  }
+
+  /**
+   * Test that script exists with non-zero exit code when command fails.
+   * @throws IOException
+   */
+  @Test
+  public void testShellScriptBuilderWithNoRedirection() throws IOException {
+    ShellScriptBuilder builder = ShellScriptBuilder.create();
+
+    Path logDir = new Path(localLogDir.getAbsolutePath());
+    File stdout = new File(logDir.toString(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDOUT);
+    File stderr = new File(logDir.toString(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDERR);
+
+    //should redirect to specified stdout path
+    String TEST_STDOUT_ECHO = "Test stdout redirection";
+    builder.echo(TEST_STDOUT_ECHO);
+    //should fail and redirect to stderr
+    builder.mkdir(new Path("/invalidSrcDir"));
+
+    builder.command(Arrays.asList(new String[]{"unknownCommand"}));
+
+    File shellFile = Shell.appendScriptExtension(tmpDir, "testShellScriptBuilderStdOutandErrRedirection");
+    PrintStream writer = new PrintStream(new FileOutputStream(shellFile));
+    builder.write(writer);
+    writer.close();
+    try {
+      FileUtil.setExecutable(shellFile, true);
+
+      Shell.ShellCommandExecutor shexc = new Shell.ShellCommandExecutor(
+          new String[]{shellFile.getAbsolutePath()}, tmpDir);
+      try {
+        shexc.execute();
+        fail("builder shell command was expected to throw");
+      } catch (IOException e) {
+        // expected
+        System.out.println("Received an expected exception: " + e.getMessage());
+
+        Assert.assertEquals(false, stdout.exists());
+        Assert.assertEquals(false, stderr.exists());
+      }
+    } finally {
+      FileUtil.fullyDelete(shellFile);
+    }
+  }
   /*
    * ${foo.version} is substituted to suffix a specific version number
    */