You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tez.apache.org by "Jeff Zhang (JIRA)" <ji...@apache.org> on 2016/02/18 05:51:18 UTC

[jira] [Commented] (TEZ-3124) Running task hangs due to missing event to initialize input in recovery

    [ https://issues.apache.org/jira/browse/TEZ-3124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15151735#comment-15151735 ] 

Jeff Zhang commented on TEZ-3124:
---------------------------------

tasksNotYetScheduled is accessed by both dispatcher thread and VM thread, protected by writelock is not sufficient. Should change it to AtomicBoolean. 

> Running task hangs due to missing event to initialize input in recovery
> -----------------------------------------------------------------------
>
>                 Key: TEZ-3124
>                 URL: https://issues.apache.org/jira/browse/TEZ-3124
>             Project: Apache Tez
>          Issue Type: Bug
>    Affects Versions: 0.8.2
>            Reporter: Jeff Zhang
>            Assignee: Jeff Zhang
>              Labels: Recovery
>             Fix For: 0.8.3
>
>         Attachments: a.log
>
>
> {noformat}
> 2016-02-09 04:48:42 Starting to run new task attempt: attempt_1454993155302_0001_1_00_000061_3
> /attempt_1454993155302_0001_1_00_000061
> 2016-02-09 04:48:43,196 [INFO] [I/O Setup 0 Initialize: {MRInput}] |input.MRInput|: MRInput using newmapreduce API=true, split via event=true, numPhysicalInputs=1
> 2016-02-09 04:48:43,200 [INFO] [I/O Setup 0 Initialize: {MRInput}] |input.MRInputLegacy|: MRInput MRInputLegacy deferring initialization
> 2016-02-09 04:48:43,333 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: Initialized processor
> 2016-02-09 04:48:43,333 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: Waiting for 2 initializers to finish
> 2016-02-09 04:48:43,333 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: Waiting for 1 initializers to finish
> 2016-02-09 04:48:43,333 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: All initializers finished
> 2016-02-09 04:48:43,345 [INFO] [TezChild] |resources.MemoryDistributor|: InitialRequests=[MRInput:INPUT:0:org.apache.tez.mapreduce.input.MRInputLegacy], [ireduce1:OUTPUT:1802502144:org.apache.tez.runtime.library.output.OrderedPartitionedKVOutput]
> 2016-02-09 04:48:43,559 [INFO] [TezChild] |resources.WeightedScalingMemoryDistributor|: ScaleRatiosUsed=[PARTITIONED_UNSORTED_OUTPUT:1][UNSORTED_OUTPUT:1][UNSORTED_INPUT:1][SORTED_OUTPUT:12][SORTED_MERGED_INPUT:12][PROCESSOR:1][OTHER:1]
> 2016-02-09 04:48:43,563 [INFO] [TezChild] |resources.WeightedScalingMemoryDistributor|: InitialReservationFraction=0.3, AdditionalReservationFractionForIOs=0.03, finalReserveFractionUsed=0.32999999999999996
> 2016-02-09 04:48:43,564 [INFO] [TezChild] |resources.WeightedScalingMemoryDistributor|: Scaling Requests. NumRequests: 2, numScaledRequests: 13, TotalRequested: 1802502144, TotalRequestedScaled: 1.663848132923077E9, TotalJVMHeap: 2577399808, TotalAvailable: 1726857871, TotalRequested/TotalJVMHeap:0.70
> 2016-02-09 04:48:43,564 [INFO] [TezChild] |resources.MemoryDistributor|: Allocations=[MRInput:org.apache.tez.mapreduce.input.MRInputLegacy:INPUT:0:0], [ireduce1:org.apache.tez.runtime.library.output.OrderedPartitionedKVOutput:OUTPUT:1802502144:1726857871]
> 2016-02-09 04:48:43,564 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: Starting Inputs/Outputs
> 2016-02-09 04:48:43,572 [INFO] [I/O Setup 1 Start: {MRInput}] |runtime.LogicalIOProcessorRuntimeTask|: Started Input with src edge: MRInput
> 2016-02-09 04:48:43,572 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: Input: MRInput being auto started by the framework. Subsequent instances will not be auto-started
> 2016-02-09 04:48:43,573 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: Num IOs determined for AutoStart: 1
> 2016-02-09 04:48:43,574 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: Waiting for 1 IOs to start
> 2016-02-09 04:48:43,574 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: AutoStartComplete
> 2016-02-09 04:48:43,583 [INFO] [TezChild] |task.TaskRunner2Callable|: Running task, taskAttemptId=attempt_1454993155302_0001_1_00_000061_3
> 2016-02-09 04:48:43,583 [INFO] [TezChild] |map.MapProcessor|: Running map: attempt_1454993155302_0001_1_00_000061_3_10001
> 2016-02-09 04:48:43,675 [INFO] [TezChild] |impl.ExternalSorter|: ireduce1 using: memoryMb=1646, keySerializerClass=class org.apache.hadoop.io.IntWritable, valueSerializerClass=org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer@5f143de6, comparator=org.apache.hadoop.io.IntWritable$Comparator@ec52d1f, partitioner=org.apache.tez.mapreduce.partition.MRPartitioner, serialization=org.apache.hadoop.io.serializer.WritableSerialization
> 2016-02-09 04:48:43,686 [INFO] [TezChild] |impl.PipelinedSorter|: Setting up PipelinedSorter for ireduce1: , UsingHashComparator=false
> 2016-02-09 04:48:45,093 [INFO] [TezChild] |impl.PipelinedSorter|: Newly allocated block size=1725956096, index=0, Number of buffers=1, currentAllocatableMemory=0, currentBufferSize=1725956096, total=1725956096
> 2016-02-09 04:48:45,093 [INFO] [TezChild] |impl.PipelinedSorter|: Pre allocating rest of memory buffers upfront
> 2016-02-09 04:48:45,093 [INFO] [TezChild] |impl.PipelinedSorter|: Setting up PipelinedSorter for ireduce1: , UsingHashComparator=false#blocks=1, maxMemUsage=1725956096, lazyAllocateMem=false, minBlockSize=2097152000, initial BLOCK_SIZE=1725956096, finalMergeEnabled=true, pipelinedShuffle=false, sendEmptyPartitions=true, tez.runtime.io.sort.mb=1646
> 2016-02-09 04:48:45,099 [INFO] [TezChild] |impl.PipelinedSorter|: ireduce1: reserved.remaining()=1725956096, reserved.metasize=16777216
> 2016-02-09 04:48:45,175 [INFO] [TezChild] |input.MRInput|: Initialized MRInput: MRInput
> 2016-02-09 08:55:40,790 [INFO] [TaskHeartbeatThread] |task.TaskReporter|: Received should die response from AM
> 2016-02-09 08:55:40,791 [INFO] [TaskHeartbeatThread] |task.TaskReporter|: Asked to die via task heartbeat
> 2016-02-09 08:55:40,796 [INFO] [TaskHeartbeatThread] |task.TezTaskRunner2|: Attempting to abort attempt_1454993155302_0001_1_00_000061_3 due to an invocation of shutdownRequested
> 2016-02-09 08:55:40,804 [INFO] [TezChild] |task.TaskRunner2Callable|: Cleaning up task attempt_1454993155302_0001_1_00_000061_3, stopRequested=true
> 2016-02-09 08:55:40,805 [INFO] [TezChild] |runtime.LogicalIOProcessorRuntimeTask|: Final Counters for attempt_1454993155302_0001_1_00_000061_3: Counters: 20 [[org.apache.tez.common.counters.TaskCounter SPILLED_RECORDS=0, INPUT_RECORDS_PROCESSED=0, OUTPUT_RECORDS=0, OUTPUT_BYTES=0, OUTPUT_BYTES_WITH_OVERHEAD=0, OUTPUT_BYTES_PHYSICAL=0, ADDITIONAL_SPILLS_BYTES_WRITTEN=0, ADDITIONAL_SPILLS_BYTES_READ=0, ADDITIONAL_SPILL_COUNT=0, SHUFFLE_CHUNK_COUNT=0][TaskCounter_map_INPUT_MRInput INPUT_RECORDS_PROCESSED=0][TaskCounter_map_OUTPUT_ireduce1 ADDITIONAL_SPILLS_BYTES_READ=0, ADDITIONAL_SPILLS_BYTES_WRITTEN=0, ADDITIONAL_SPILL_COUNT=0, OUTPUT_BYTES=0, OUTPUT_BYTES_PHYSICAL=0, OUTPUT_BYTES_WITH_OVERHEAD=0, OUTPUT_RECORDS=0, SHUFFLE_CHUNK_COUNT=0, SPILLED_RECORDS=0]]
> {noformat}
> It is caused by race issue between dispatcher thread and VM thread. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)