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

David Mollitor commented on HIVE-24484:
---------------------------------------

OK. When there is Hive+Tez and a LIMIT clause, each Tez Vertex is "interrupted" 
to signal that it should stop running when the LIMIT is reached.  
[HADOOP-17313] adds a lock into the FileSystem API that throws an 
{{InterruptedIOException}} if the thread is interrupted (and clears the 
interrupt flag).  Tez sees this exception as a failure and reports an error.  
Tez probably needs to be updated to handle this situation, but it ain't fun.

https://github.com/apache/hadoop/blob/a3b9c37a397ad4188041dd80621bdeefc46885f2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java#L3556-L3560

{code:none}
Error while running task ( failure ) : java.lang.RuntimeException: 
java.io.IOException: java.io.IOException: java.io.InterruptedIOException: 
java.lang.InterruptedException
 at 
org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:206)
 at 
org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.<init>(TezGroupedSplitsInputFormat.java:145)
 at 
org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat.getRecordReader(TezGroupedSplitsInputFormat.java:111)
 at 
org.apache.tez.mapreduce.lib.MRReaderMapred.setupOldRecordReader(MRReaderMapred.java:156)
 at org.apache.tez.mapreduce.lib.MRReaderMapred.<init>(MRReaderMapred.java:75)
 at 
org.apache.tez.mapreduce.input.MultiMRInput.initFromEvent(MultiMRInput.java:196)
 at 
org.apache.tez.mapreduce.input.MultiMRInput.handleEvents(MultiMRInput.java:154)
 at 
org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.handleEvent(LogicalIOProcessorRuntimeTask.java:739)
 at 
org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.access$600(LogicalIOProcessorRuntimeTask.java:108)
 at 
org.apache.tez.runtime.LogicalIOProcessorRuntimeTask$1.runInternal(LogicalIOProcessorRuntimeTask.java:816)
 at org.apache.tez.common.RunnableWithNdc.run(RunnableWithNdc.java:35)
 at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.IOException: java.io.IOException: 
java.io.InterruptedIOException: java.lang.InterruptedException
 at 
org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97)
 at 
org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57)
 at 
org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:422)
 at 
org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:203)
 ... 11 more
Caused by: java.io.IOException: java.io.InterruptedIOException: 
java.lang.InterruptedException
 at 
org.apache.hadoop.hive.llap.io.api.impl.LlapInputFormat.getRecordReader(LlapInputFormat.java:141)
 at 
org.apache.hadoop.hive.ql.io.RecordReaderWrapper.create(RecordReaderWrapper.java:72)
 at 
org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:419)
 ... 12 more
Caused by: java.io.InterruptedIOException: java.lang.InterruptedException
 at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3559)
 at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
 at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
 at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
 at org.apache.hadoop.mapred.LineRecordReader.<init>(LineRecordReader.java:111)
 at 
org.apache.hadoop.mapred.TextInputFormat.getRecordReader(TextInputFormat.java:67)
 at 
org.apache.hadoop.hive.llap.io.api.impl.LlapInputFormat.getRecordReader(LlapInputFormat.java:123)
 ... 14 more
Caused by: java.lang.InterruptedException
 at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1306)
 at java.util.concurrent.Semaphore.acquire(Semaphore.java:312)
 at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3556)
 ... 20 more
{code}

> Upgrade Hadoop to 3.3.1
> -----------------------
>
>                 Key: HIVE-24484
>                 URL: https://issues.apache.org/jira/browse/HIVE-24484
>             Project: Hive
>          Issue Type: Improvement
>            Reporter: David Mollitor
>            Assignee: David Mollitor
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 2h 23m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to