Selaa lähdekoodia

YARN-814. Improving diagnostics when containers fail during launch due to various reasons like invalid env etc. Contributed by Jian He.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1504732 13f79535-47bb-0310-9956-ffa450edef68
Vinod Kumar Vavilapalli 12 vuotta sitten
vanhempi
commit
7a29bccd7a

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

@@ -74,6 +74,9 @@ Release 2.1.1-beta - UNRELEASED
     YARN-62. Modified NodeManagers to avoid AMs from abusing container tokens for
     repetitive container launches. (Omkar Vinit Joshi via vinodkv)
 
+    YARN-814. Improving diagnostics when containers fail during launch due to
+    various reasons like invalid env etc. (Jian He via vinodkv)
+
 Release 2.1.0-beta - 2013-07-02
 
   INCOMPATIBLE CHANGES

+ 11 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java

@@ -39,8 +39,10 @@ import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
 import org.apache.hadoop.yarn.server.nodemanager.util.ProcessIdFileReader;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
 
 public abstract class ContainerExecutor implements Configurable {
 
@@ -291,15 +293,16 @@ public abstract class ContainerExecutor implements Configurable {
   }
 
   public static class DelayedProcessKiller extends Thread {
+    private Container container;
     private final String user;
     private final String pid;
     private final long delay;
     private final Signal signal;
     private final ContainerExecutor containerExecutor;
 
-    public DelayedProcessKiller(String user, String pid, long delay,
-        Signal signal,
-        ContainerExecutor containerExecutor) {
+    public DelayedProcessKiller(Container container, String user, String pid,
+        long delay, Signal signal, ContainerExecutor containerExecutor) {
+      this.container = container;
       this.user = user;
       this.pid = pid;
       this.delay = delay;
@@ -316,7 +319,11 @@ public abstract class ContainerExecutor implements Configurable {
       } catch (InterruptedException e) {
         return;
       } catch (IOException e) {
-        LOG.warn("Exception when killing task " + pid, e);
+        String message = "Exception when user " + user + " killing task " + pid
+            + " in DelayedProcessKiller: " + StringUtils.stringifyException(e);
+        LOG.warn(message);
+        container.handle(new ContainerDiagnosticsUpdateEvent(container
+          .getContainerId(), message));
       }
     }
   }

+ 18 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ExitCodeException;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -203,11 +204,23 @@ public class DefaultContainerExecutor extends ContainerExecutor {
         return -1;
       }
       int exitCode = shExec.getExitCode();
-      LOG.warn("Exit code from task is : " + exitCode);
-      String message = shExec.getOutput();
-      logOutput(message);
-      container.handle(new ContainerDiagnosticsUpdateEvent(containerId,
-          message));
+      LOG.warn("Exit code from container " + containerId + " is : " + exitCode);
+      // 143 (SIGTERM) and 137 (SIGKILL) exit codes means the container was
+      // terminated/killed forcefully. In all other cases, log the
+      // container-executor's output
+      if (exitCode != ExitCode.FORCE_KILLED.getExitCode()
+          && exitCode != ExitCode.TERMINATED.getExitCode()) {
+        LOG.warn("Exception from container-launch with container ID: "
+            + containerId + " and exit code: " + exitCode , e);
+        logOutput(shExec.getOutput());
+        String diagnostics = "Exception from container-launch: \n"
+            + StringUtils.stringifyException(e) + "\n" + shExec.getOutput();
+        container.handle(new ContainerDiagnosticsUpdateEvent(containerId,
+            diagnostics));
+      } else {
+        container.handle(new ContainerDiagnosticsUpdateEvent(containerId,
+            "Container killed on request. Exit code is " + exitCode));
+      }
       return exitCode;
     } finally {
       ; //

+ 20 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -146,7 +146,8 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       shExec.execute();
     } catch (ExitCodeException e) {
       int exitCode = shExec.getExitCode();
-      LOG.warn("Exit code from container is : " + exitCode);
+      LOG.warn("Exit code from container executor initialization is : "
+          + exitCode, e);
       logOutput(shExec.getOutput());
       throw new IOException("Linux container executor not configured properly"
           + " (error=" + exitCode + ")", e);
@@ -203,10 +204,11 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       }
     } catch (ExitCodeException e) {
       int exitCode = shExec.getExitCode();
-      LOG.warn("Exit code from container is : " + exitCode);
+      LOG.warn("Exit code from container " + locId + " startLocalizer is : "
+          + exitCode, e);
       logOutput(shExec.getOutput());
-      throw new IOException("App initialization failed (" + exitCode + 
-          ") with output: " + shExec.getOutput(), e);
+      throw new IOException("Application " + appId + " initialization failed" +
+      		" (exitCode=" + exitCode + ") with output: " + shExec.getOutput(), e);
     }
   }
 
@@ -255,19 +257,18 @@ public class LinuxContainerExecutor extends ContainerExecutor {
         return ExitCode.TERMINATED.getExitCode();
       }
     } catch (ExitCodeException e) {
-
       if (null == shExec) {
         return -1;
       }
-
       int exitCode = shExec.getExitCode();
-      LOG.warn("Exit code from container is : " + exitCode);
+      LOG.warn("Exit code from container " + containerId + " is : " + exitCode);
       // 143 (SIGTERM) and 137 (SIGKILL) exit codes means the container was
       // terminated/killed forcefully. In all other cases, log the
       // container-executor's output
       if (exitCode != ExitCode.FORCE_KILLED.getExitCode()
           && exitCode != ExitCode.TERMINATED.getExitCode()) {
-        LOG.warn("Exception from container-launch : ", e);
+        LOG.warn("Exception from container-launch with container ID: "
+            + containerId + " and exit code: " + exitCode , e);
         logOutput(shExec.getOutput());
         String diagnostics = "Exception from container-launch: \n"
             + StringUtils.stringifyException(e) + "\n" + shExec.getOutput();
@@ -309,9 +310,12 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       if (ret_code == ResultCode.INVALID_CONTAINER_PID.getValue()) {
         return false;
       }
+      LOG.warn("Error in signalling container " + pid + " with " + signal
+          + "; exit = " + ret_code, e);
       logOutput(shExec.getOutput());
-      throw new IOException("Problem signalling container " + pid + " with " +
-                            signal + "; exit = " + ret_code);
+      throw new IOException("Problem signalling container " + pid + " with "
+          + signal + "; output: " + shExec.getOutput() + " and exitCode: "
+          + ret_code, e);
     }
     return true;
   }
@@ -345,13 +349,10 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       }
     } catch (IOException e) {
       int exitCode = shExec.getExitCode();
-      LOG.warn("Exit code from container is : " + exitCode);
-      if (exitCode != 0) {
-        LOG.error("DeleteAsUser for " + dir.toUri().getPath()
-            + " returned with non-zero exit code" + exitCode);
-        LOG.error("Output from LinuxContainerExecutor's deleteAsUser follows:");
-        logOutput(shExec.getOutput());
-      }
+      LOG.error("DeleteAsUser for " + dir.toUri().getPath()
+          + " returned with exit code: " + exitCode, e);
+      LOG.error("Output from LinuxContainerExecutor's deleteAsUser follows:");
+      logOutput(shExec.getOutput());
     }
   }
   
@@ -372,9 +373,10 @@ public class LinuxContainerExecutor extends ContainerExecutor {
         shExec.execute();
     } catch (IOException e) {
         int ret_code = shExec.getExitCode();
+        LOG.warn("Exception in LinuxContainerExecutor mountCgroups ", e);
         logOutput(shExec.getOutput());
         throw new IOException("Problem mounting cgroups " + cgroupKVs + 
-                  "; exit code = " + ret_code, e);
+          "; exit code = " + ret_code + " and output: " + shExec.getOutput(), e);
     }
   }  
 }

+ 9 - 3
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

@@ -61,6 +61,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
@@ -308,6 +309,7 @@ public class ContainerLaunch implements Callable<Integer> {
    * the process id is available.
    * @throws IOException
    */
+  @SuppressWarnings("unchecked") // dispatcher not typed
   public void cleanupContainer() throws IOException {
     ContainerId containerId = container.getContainerId();
     String containerIdStr = ConverterUtils.toString(containerId);
@@ -355,13 +357,17 @@ public class ContainerLaunch implements Callable<Integer> {
               + " as user " + user
               + " for container " + containerIdStr
               + ", result=" + (result? "success" : "failed"));
-          new DelayedProcessKiller(user,
+          new DelayedProcessKiller(container, user,
               processId, sleepDelayBeforeSigKill, Signal.KILL, exec).start();
         }
       }
     } catch (Exception e) {
-      LOG.warn("Got error when trying to cleanup container " + containerIdStr
-          + ", error=" + e.getMessage());
+      String message =
+          "Exception when trying to cleanup container " + containerIdStr
+              + ": " + StringUtils.stringifyException(e);
+      LOG.warn(message);
+      dispatcher.getEventHandler().handle(
+        new ContainerDiagnosticsUpdateEvent(containerId, message));
     } finally {
       // cleanup pid file if present
       if (pidFilePath != null) {

+ 170 - 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

@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.Shell.ExitCodeException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
@@ -98,7 +99,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       tempFile = Shell.appendScriptExtension(tmpDir, "temp");
       String timeoutCommand = Shell.WINDOWS ? "@echo \"hello\"" :
         "echo \"hello\"";
-      PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));    
+      PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));
       FileUtil.setExecutable(shellFile, true);
       writer.println(timeoutCommand);
       writer.close();
@@ -132,7 +133,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       assertEquals(shexc.getExitCode(), 0);
       assert(shexc.getOutput().contains("hello"));
 
-      symLinkFile = new File(tmpDir, badSymlink);      
+      symLinkFile = new File(tmpDir, badSymlink);
     }
     finally {
       // cleanup
@@ -151,6 +152,173 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     }
   }
 
+  // test the diagnostics are generated
+  @Test (timeout = 20000)
+  public void testInvalidSymlinkDiagnostics() throws IOException  {
+
+    File shellFile = null;
+    File tempFile = null;
+    String symLink = Shell.WINDOWS ? "test.cmd" :
+      "test";
+    File symLinkFile = null;
+
+    try {
+      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));
+      FileUtil.setExecutable(shellFile, true);
+      writer.println(timeoutCommand);
+      writer.close();
+
+      Map<Path, List<String>> resources =
+          new HashMap<Path, List<String>>();
+      //This is an invalid path and should throw exception because of No such file.
+      Path invalidPath = new Path(shellFile.getAbsolutePath()+"randomPath");
+      resources.put(invalidPath, Arrays.asList(symLink));
+      FileOutputStream fos = new FileOutputStream(tempFile);
+
+      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("\"" + symLink + "\"");
+      } else {
+        commands.add("/bin/sh ./\\\"" + symLink + "\\\"");
+      }
+      ContainerLaunch.writeLaunchEnv(fos, env, resources, commands);
+      fos.flush();
+      fos.close();
+      FileUtil.setExecutable(tempFile, true);
+
+      Shell.ShellCommandExecutor shexc
+      = new Shell.ShellCommandExecutor(new String[]{tempFile.getAbsolutePath()}, tmpDir);
+      String diagnostics = null;
+      try {
+        shexc.execute();
+        Assert.fail("Should catch exception");
+      } catch(ExitCodeException e){
+        diagnostics = e.getMessage();
+      }
+      Assert.assertNotNull(diagnostics);
+      Assert.assertTrue(shexc.getExitCode() != 0);
+      symLinkFile = new File(tmpDir, symLink);
+    }
+    finally {
+      // cleanup
+      if (shellFile != null
+          && shellFile.exists()) {
+        shellFile.delete();
+      }
+      if (tempFile != null
+          && tempFile.exists()) {
+        tempFile.delete();
+      }
+      if (symLinkFile != null
+          && symLinkFile.exists()) {
+        symLinkFile.delete();
+      }
+    }
+  }
+
+  @Test (timeout = 20000)
+  public void testInvalidEnvSyntaxDiagnostics() throws IOException  {
+
+    File shellFile = null;
+    try {
+      shellFile = Shell.appendScriptExtension(tmpDir, "hello");
+      String timeoutCommand = Shell.WINDOWS ? "@echo \"hello\"" :
+        "echo \"hello\"";
+      PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));
+      FileUtil.setExecutable(shellFile, true);
+      writer.println(timeoutCommand);
+      writer.close();
+      Map<Path, List<String>> resources =
+          new HashMap<Path, List<String>>();
+      FileOutputStream fos = new FileOutputStream(shellFile);
+
+      Map<String, String> env = new HashMap<String, String>();
+      // invalid env
+      env.put(
+          "APPLICATION_WORKFLOW_CONTEXT", "{\"workflowId\":\"609f91c5cd83\"," +
+          "\"workflowName\":\"\n\ninsert table " +
+          "\npartition (cd_education_status)\nselect cd_demo_sk, cd_gender, " );
+      List<String> commands = new ArrayList<String>();
+      ContainerLaunch.writeLaunchEnv(fos, env, resources, commands);
+      fos.flush();
+      fos.close();
+
+      Shell.ShellCommandExecutor shexc
+      = new Shell.ShellCommandExecutor(new String[]{shellFile.getAbsolutePath()}, tmpDir);
+      String diagnostics = null;
+      try {
+        shexc.execute();
+        Assert.fail("Should catch exception");
+      } catch(ExitCodeException e){
+        diagnostics = e.getMessage();
+      }
+      Assert.assertTrue(diagnostics.contains("command not found"));
+      Assert.assertTrue(shexc.getExitCode() != 0);
+    }
+    finally {
+      // cleanup
+      if (shellFile != null
+          && shellFile.exists()) {
+        shellFile.delete();
+      }
+    }
+  }
+
+  @Test (timeout = 20000)
+  public void testContainerLaunchStdoutAndStderrDiagnostics() throws IOException {
+
+    File shellFile = null;
+    try {
+      shellFile = Shell.appendScriptExtension(tmpDir, "hello");
+      // echo "hello" to stdout and "error" to stderr and exit code with 2;
+      String command = Shell.WINDOWS ? "@echo \"hello\"; @echo \"error\" 1>&2; exit 2;" :
+        "echo \"hello\"; echo \"error\" 1>&2; exit 2;";
+      PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));
+      FileUtil.setExecutable(shellFile, true);
+      writer.println(command);
+      writer.close();
+      Map<Path, List<String>> resources =
+          new HashMap<Path, List<String>>();
+      FileOutputStream fos = new FileOutputStream(shellFile);
+
+      Map<String, String> env = new HashMap<String, String>();
+      List<String> commands = new ArrayList<String>();
+      commands.add(command);
+      ContainerLaunch.writeLaunchEnv(fos, env, resources, commands);
+      fos.flush();
+      fos.close();
+
+      Shell.ShellCommandExecutor shexc
+      = new Shell.ShellCommandExecutor(new String[]{shellFile.getAbsolutePath()}, tmpDir);
+      String diagnostics = null;
+      try {
+        shexc.execute();
+        Assert.fail("Should catch exception");
+      } catch(ExitCodeException e){
+        diagnostics = e.getMessage();
+      }
+      // test stderr
+      Assert.assertTrue(diagnostics.contains("error"));
+      // test stdout
+      Assert.assertTrue(shexc.getOutput().contains("hello"));
+      Assert.assertTrue(shexc.getExitCode() == 2);
+    }
+    finally {
+      // cleanup
+      if (shellFile != null
+          && shellFile.exists()) {
+        shellFile.delete();
+      }
+    }
+  }
+
   /**
    * See if environment variable is forwarded using sanitizeEnv.
    * @throws Exception