Uploaded image for project: 'Apache Tez'
  1. Apache Tez
  2. TEZ-1240 Umbrella jira tracking propagation of diagnostics for errors
  3. TEZ-1238

Display more clear diagnostics info on client side on task failures

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 0.4.0
    • 0.5.0
    • None
    • None

    Description

      I have a tez job which is failed due to that I didn't put my jar to the local resources. But on the client side, the exception is not clear for me to figure what's wrong with it. The real reason is that It couldn't load the Processor class. I have to run command "yarn logs" to find the real exception in the container logs.
      I also have another case that has exception in the my Processor, the message on the client side is still not clear to me. I think that should we pass the real exception to the diagnostics and display it in client side, this should help user to find out what's wrong with their program.

      Exception on client side

      14/06/26 14:57:15 INFO rpc.DAGClientRPCImpl: VertexStatus: VertexName:
      summer Progress: 0% TotalTasks: 1 Succeeded: 0 Running: 0 Failed: 0 Killed:
      114/06/26 14:57:15 INFO rpc.DAGClientRPCImpl: VertexStatus: VertexName:
      tokenizer Progress: 0% TotalTasks: 1 Succeeded: 0 Running: 0 Failed: 1
      Killed: 014/06/26 14:57:15 INFO rpc.DAGClientRPCImpl: DAG completed.
      FinalState=FAILEDDAG diagnostics:[Vertex failed, vertexName=tokenizer,
      vertexId=vertex_1403765612557_0004_1_00, diagnostics=[Task failed,
      taskId=task_1403765612557_0004_1_00_000000, diagnostics=[TaskAttempt 0
      failed, info=[Container container_1403765612557_0004_01_000002 COMPLETED
      with diagnostics set to [Exception from container-launch:
      org.apache.hadoop.util.Shell$ExitCodeException:
      org.apache.hadoop.util.Shell$ExitCodeException: at
      org.apache.hadoop.util.Shell.runCommand(Shell.java:505)
      
      at org.apache.hadoop.util.Shell.run(Shell.java:418)
      
      at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:650)
      
      at
      org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(
      DefaultContainerExecutor.java:195)
      
      at
      org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(
      ContainerLaunch.java:300)
      
      at
      org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(
      ContainerLaunch.java:81)
      
      at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      
      at java.util.concurrent.ThreadPoolExecutor.runWorker(
      ThreadPoolExecutor.java:1145)
      
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(
      ThreadPoolExecutor.java:615)
      
      at java.lang.Thread.run(Thread.java:745)
      
      Container exited with a non-zero exit code 1
      

      The real exception in container log:

      2014-06-26 14:57:02,146 ERROR [main]
      org.apache.hadoop.yarn.YarnUncaughtExceptionHandler: Thread
      Thread[main,5,main] threw an Exception.
      org.apache.tez.dag.api.TezUncheckedException: Unable to load class:
      com.zjffdu.tutorial.tez.WordCount$TokenProcessor
          at org.apache.tez.common.RuntimeUtils.getClazz(RuntimeUtils.java:44)
          at
      org.apache.tez.common.RuntimeUtils.createClazzInstance(RuntimeUtils.java:66)
          at
      org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.createProcessor(LogicalIOProcessorRuntimeTask.java:533)
          at
      org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.<init>(LogicalIOProcessorRuntimeTask.java:146)
          at
      org.apache.tez.runtime.task.TezTaskRunner.<init>(TezTaskRunner.java:78)
          at org.apache.tez.runtime.task.TezChild.run(TezChild.java:208)
          at org.apache.tez.runtime.task.TezChild.main(TezChild.java:363)
      

      Attachments

        1. Tez-1238.patch
          2 kB
          Jeff Zhang
        2. Tez-1238-2.patch
          7 kB
          Jeff Zhang
        3. Tez-1238-3.patch
          6 kB
          Jeff Zhang

        Issue Links

          Activity

            People

              zjffdu Jeff Zhang
              zjffdu Jeff Zhang
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: