Uploaded image for project: 'Apache Tez'
  1. Apache Tez
  2. TEZ-1929

pre-empted tasks should be marked as killed instead of failed

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 0.6.0
    • 0.6.1
    • None
    • None
    • Reviewed

    Description

      Observed this behavior 3 or 4 times

      • Ran a hive query with tez (query_17 at 10 TB scale)
      • Occasionally, Map_7 task will get into failed state in the middle of fetching data from other sources (only one task is available in Map_7).
      2015-01-08 00:19:10,289 INFO [Fetcher [Map_6] #0] impl.ShuffleManager: Completed fetch for attempt: InputAttemptIdentifier [inputIdentifier=InputIdentifier [inputIndex=0], attemptNumber=0, pathComponent=attempt_1420000126204_0233_1_06_000000_0_10003] to MEMORY, CompressedSize=6757, DecompressedSize=16490,EndTime=1420705150289, TimeTaken=5, Rate=1.29 MB/s
      2015-01-08 00:19:10,290 INFO [Fetcher [Map_6] #0] impl.ShuffleManager: All inputs fetched for input vertex : Map 6
      2015-01-08 00:19:10,290 INFO [Fetcher [Map_6] #0] impl.ShuffleManager: copy(0 of 1. Transfer rate (CumulativeDataFetched/TimeSinceInputStarted)) 0.01 MB/s)
      2015-01-08 00:19:10,290 INFO [ShuffleRunner [Map_6]] impl.ShuffleManager: Shutting down FetchScheduler, Was Interrupted: false
      2015-01-08 00:19:10,290 INFO [ShuffleRunner [Map_6]] impl.ShuffleManager: Scheduler thread completed
      2015-01-08 00:19:41,986 INFO [TaskHeartbeatThread] task.TaskReporter: Received should die response from AM
      2015-01-08 00:19:41,986 INFO [TaskHeartbeatThread] task.TaskReporter: Asked to die via task heartbeat
      2015-01-08 00:19:41,987 INFO [main] task.TezTaskRunner: Interrupted while waiting for task to complete. Interrupting task
      2015-01-08 00:19:41,987 INFO [main] task.TezTaskRunner: Shutdown requested... returning
      2015-01-08 00:19:41,987 INFO [main] task.TezChild: Got a shouldDie notification via hearbeats. Shutting down
      2015-01-08 00:19:41,990 ERROR [TezChild] tez.TezProcessor: java.lang.InterruptedException
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2048)
      	at org.apache.tez.runtime.InputReadyTracker$InputReadyMonitor.awaitCondition(InputReadyTracker.java:120)
      	at org.apache.tez.runtime.InputReadyTracker.waitForAnyInputReady(InputReadyTracker.java:83)
      	at org.apache.tez.runtime.api.impl.TezProcessorContextImpl.waitForAnyInputReady(TezProcessorContextImpl.java:106)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:153)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:328)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      

      From the initial look, it appears that TaskAttemptListenerImpTezDag.heartbeat is unable to identify the containerId from registeredContainers. Need to verify this.

      I will attach the sample task log and the tez-ui details.

      Attachments

        1. TEZ-1929.3.patch
          8 kB
          Bikas Saha
        2. TEZ-1929.2.patch
          8 kB
          Bikas Saha
        3. TEZ-1929.1.patch
          3 kB
          Bikas Saha
        4. applog.txt.gz
          1.37 MB
          Rajesh Balamohan
        5. tasklog.txt
          22 kB
          Rajesh Balamohan
        6. Screen Shot 2015-01-08 at 2.09.11 PM.png
          189 kB
          Rajesh Balamohan
        7. Screen Shot 2015-01-08 at 2.28.04 PM.png
          408 kB
          Rajesh Balamohan

        Activity

          People

            bikassaha Bikas Saha
            rajesh.balamohan Rajesh Balamohan
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: