You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tez.apache.org by "Jonathan Eagles (JIRA)" <ji...@apache.org> on 2017/09/13 21:53:00 UTC

[jira] [Commented] (TEZ-3831) Reduce Unordered memory needed for storing empty completed events

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

Jonathan Eagles commented on TEZ-3831:
--------------------------------------

From the below snippet of the task log, we can see that the UnorderedKVReader was doing a blocking take from the completedInputs, while the ShuffleManager was fetching, and adding empty completes to the completedInputs queue. While 11 million empty  partitions had been fetched only 8 million were in the completedInputs queue.
{noformat}
2017-09-12 22:01:11,656 [INFO] [TezTaskEventRouter{attempt_1502569884713_1043511_1_05_000000_0}] |impl.ShuffleInputEventHandlerImpl|: scope-416: numDmeEventsSeen=11297691, numDmeEventsSeenWithNoData=11276605, numObsoletionEventsSeen=0
2017-09-12 22:01:01,531 [ERROR] [Fetcher_B {scope_416} #19] |impl.ShuffleManager|: scope_416: Fetcher failed with error:
java.lang.OutOfMemoryError: GC overhead limit exceeded
        at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.&lt;init&gt;(IFileInputStream.java:88)
        at org.apache.tez.runtime.library.common.sort.impl.IFile$Reader.readToMemory(IFile.java:617)
        at org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.shuffleToMemory(ShuffleUtils.java:121)
        at org.apache.tez.runtime.library.common.shuffle.Fetcher.fetchInputs(Fetcher.java:950)
        at org.apache.tez.runtime.library.common.shuffle.Fetcher.doHttpFetch(Fetcher.java:599)
        at org.apache.tez.runtime.library.common.shuffle.Fetcher.doHttpFetch(Fetcher.java:486)
        at org.apache.tez.runtime.library.common.shuffle.Fetcher.callInternal(Fetcher.java:284)
        at org.apache.tez.runtime.library.common.shuffle.Fetcher.callInternal(Fetcher.java:76)
        at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
2017-09-12 22:01:42,524 [INFO] [TaskHeartbeatThread] |task.TezTaskRunner2|: Attempting to abort attempt_1502569884713_1043511_1_05_000000_0 due to an invocation of umbilicalFatalError
2017-09-12 22:01:46,941 [WARN] [TezChild] |readers.UnorderedKVReader|: Interrupted while waiting for next available input
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 java.util.concurrent.LinkedBlockingDeque.takeFirst(LinkedBlockingDeque.java:492)
        at java.util.concurrent.LinkedBlockingDeque.take(LinkedBlockingDeque.java:680)
        at org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager.getNextInput(ShuffleManager.java:893)
        at org.apache.tez.runtime.library.common.readers.UnorderedKVReader.moveToNextInput(UnorderedKVReader.java:202)
        at org.apache.tez.runtime.library.common.readers.UnorderedKVReader.next(UnorderedKVReader.java:125)
        at org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.collectSample(PigProcessor.java:417)
        at org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.initializeInputs(PigProcessor.java:320)
        at org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.run(PigProcessor.java:228)
        at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
        at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
        at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1936)
        at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
        at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
        at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
2017-09-12 22:01:46,941 [ERROR] [TezChild] |runtime.PigProcessor|: Encountered exception while processing:
org.apache.tez.runtime.library.api.IOInterruptedException: java.lang.InterruptedException
        at org.apache.tez.runtime.library.common.readers.UnorderedKVReader.moveToNextInput(UnorderedKVReader.java:206)
        at org.apache.tez.runtime.library.common.readers.UnorderedKVReader.next(UnorderedKVReader.java:125)
        at org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.collectSample(PigProcessor.java:417)
        at org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.initializeInputs(PigProcessor.java:320)
        at org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.run(PigProcessor.java:228)
        at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
        at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
        at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1936)
        at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
        at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
        at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
Caused by: 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 java.util.concurrent.LinkedBlockingDeque.takeFirst(LinkedBlockingDeque.java:492)
        at java.util.concurrent.LinkedBlockingDeque.take(LinkedBlockingDeque.java:680)
        at org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager.getNextInput(ShuffleManager.java:893)
        at org.apache.tez.runtime.library.common.readers.UnorderedKVReader.moveToNextInput(UnorderedKVReader.java:202)
        ... 17 more
{noformat}

> Reduce Unordered memory needed for storing empty completed events
> -----------------------------------------------------------------
>
>                 Key: TEZ-3831
>                 URL: https://issues.apache.org/jira/browse/TEZ-3831
>             Project: Apache Tez
>          Issue Type: Bug
>            Reporter: Jonathan Eagles
>            Assignee: Jonathan Eagles
>
> the completedInputs blocking queue is used to store inputs for the UnorderedKVReader to consume. With Auto-reduce parallelism enabled and nearly all empty inputs, the reader can't prune the empty events from the blocking queue fast enough to keep up. In my scenario, an OOM occurred. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)