hive-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Gopal V (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HIVE-11264) LLAP: PipelineSorter got stuck
Date Wed, 15 Jul 2015 17:10:05 GMT

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

Gopal V commented on HIVE-11264:
--------------------------------

No, that's not waiting on a Future - that's producing a Future.

The Future.wait would be in the JStack from before the kill was kicked off.

> LLAP: PipelineSorter got stuck
> ------------------------------
>
>                 Key: HIVE-11264
>                 URL: https://issues.apache.org/jira/browse/HIVE-11264
>             Project: Hive
>          Issue Type: Sub-task
>            Reporter: Sergey Shelukhin
>            Assignee: Gopal V
>
> Saving here for now, in case someone seems something similar or has a sudden insight.
> On parallel query workload, at some point, one query became stuck while everything else
finished. The query ended with 3 reducer stages, 1 vertex each. The only things running were
Reducer 3 and Reducer 2 (10-machine cluster), 3 waiting for 2, and 2 stuck for 20 minutes.
> Unfortunately log level was WARN so there's not much in terms of logs.
> When LLAP cluster was stopped, the thread running reducer 2 died like so: {noformat}
> 2015-07-14 19:02:20,136 [TezTaskRunner_attempt_1435700346116_1889_1_06_000000_104(attempt_1435700346116_1889_1_06_000000_104)]
ERROR org.apache.hadoop.hive.ql.exec.MapJoinOperator: Unexpected exception from MapJoinOperator
: java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.FutureTask@580b9a95
rejected from java.util.concurrent.ThreadPoolExecutor@61d1d95b[Terminated, pool size = 0,
active threads = 0, queued tasks = 0, completed tasks = 753]
> org.apache.hadoop.hive.ql.metadata.HiveException: java.util.concurrent.RejectedExecutionException:
Task java.util.concurrent.FutureTask@580b9a95 rejected from java.util.concurrent.ThreadPoolExecutor@61d1d95b[Terminated,
pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 753]
>    at org.apache.hadoop.hive.ql.exec.ReduceSinkOperator.process(ReduceSinkOperator.java:402)
>    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:872)
>    at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.internalForward(CommonJoinOperator.java:643)
>    at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:656)
>    at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:659)
>    at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.checkAndGenObject(CommonJoinOperator.java:755)
>    at org.apache.hadoop.hive.ql.exec.MapJoinOperator.process(MapJoinOperator.java:415)
>    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:872)
>    at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.internalForward(CommonJoinOperator.java:643)
>    at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:656)
>    at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:659)
>    at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.checkAndGenObject(CommonJoinOperator.java:755)
>    at org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator.joinObject(CommonMergeJoinOperator.java:309)
>    at org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator.joinOneGroup(CommonMergeJoinOperator.java:272)
>    at org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator.joinOneGroup(CommonMergeJoinOperator.java:265)
>    at org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator.joinFinalLeftData(CommonMergeJoinOperator.java:462)
>    at org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator.closeOp(CommonMergeJoinOperator.java:383)
>    at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:651)
>    at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.close(ReduceRecordProcessor.java:317)
>    at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:172)
>    at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:146)
>    at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:349)
>    at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:71)
>    at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:60)
>    at java.security.AccessController.doPrivileged(Native Method)
>    at javax.security.auth.Subject.doAs(Subject.java:415)
>    at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1654)
>    at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:60)
>    at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:35)
>    at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>    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)
> Caused by: java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.FutureTask@580b9a95
rejected from java.util.concurrent.ThreadPoolExecutor@61d1d95b[Terminated, pool size = 0,
active threads = 0, queued tasks = 0, completed tasks = 753]
>    at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2048)
>    at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:821)
>    at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1372)
>    at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:132)
>    at org.apache.tez.runtime.library.common.sort.impl.PipelinedSorter.sort(PipelinedSorter.java:251)
>    at org.apache.tez.runtime.library.common.sort.impl.PipelinedSorter.collect(PipelinedSorter.java:288)
>    at org.apache.tez.runtime.library.common.sort.impl.PipelinedSorter.write(PipelinedSorter.java:262)
>    at org.apache.tez.runtime.library.output.OrderedPartitionedKVOutput$1.write(OrderedPartitionedKVOutput.java:164)
>    at org.apache.hadoop.hive.ql.exec.tez.TezProcessor$TezKVOutputCollector.collect(TezProcessor.java:217)
>    at org.apache.hadoop.hive.ql.exec.ReduceSinkOperator.collect(ReduceSinkOperator.java:541)
>    at org.apache.hadoop.hive.ql.exec.ReduceSinkOperator.process(ReduceSinkOperator.java:385)
>    ... 33 more
> {noformat}
> Looks like PipelineSorter either got stuck in execution, or some other things were in
the thread pool and stuck (again note no other task was running on the machine), or something.



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

Mime
View raw message