You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "zhihai xu (JIRA)" <ji...@apache.org> on 2016/07/16 05:26:20 UTC

[jira] [Updated] (HIVE-14258) Reduce task timed out because CommonJoinOperator.genUniqueJoinObject took too long to finish without reporting progress

     [ https://issues.apache.org/jira/browse/HIVE-14258?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

zhihai xu updated HIVE-14258:
-----------------------------
    Status: Patch Available  (was: Open)

> Reduce task timed out because CommonJoinOperator.genUniqueJoinObject took too long to finish without reporting progress
> -----------------------------------------------------------------------------------------------------------------------
>
>                 Key: HIVE-14258
>                 URL: https://issues.apache.org/jira/browse/HIVE-14258
>             Project: Hive
>          Issue Type: Bug
>          Components: Query Processor
>    Affects Versions: 2.1.0
>            Reporter: zhihai xu
>            Assignee: zhihai xu
>         Attachments: HIVE-14258.patch
>
>
> Reduce task timed out because CommonJoinOperator.genUniqueJoinObject took too long to finish without reporting progress.
> This timeout happened when reducer.close() is called in ReduceTask.java.
> CommonJoinOperator.genUniqueJoinObject() called by reducer.close() will loop over every row in the AbstractRowContainer. This can take a long time if there are a large number or rows, and during this time, it does not report progress. If this runs for long enough more than "mapreduce.task.timeout", ApplicationMaster will kill the task for failing to report progress.
> we configured "mapreduce.task.timeout" as 10 minutes. I captured the stack trace in the 10 minutes before AM killed the reduce task at 2016-07-15 07:19:11.
> The following three stack traces can prove it:
> at 2016-07-15 07:09:42:
> {code}
> "main" prio=10 tid=0x00007f90ec017000 nid=0xd193 runnable [0x00007f90f62e5000]
>    java.lang.Thread.State: RUNNABLE
>         at java.io.FileInputStream.readBytes(Native Method)
>         at java.io.FileInputStream.read(FileInputStream.java:272)
>         at org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileInputStream.read(RawLocalFileSystem.java:154)
>         at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
>         at java.io.BufferedInputStream.read1(BufferedInputStream.java:275)
>         at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
>         - locked <0x00000007deecefb0> (a org.apache.hadoop.fs.BufferedFSInputStream)
>         at java.io.DataInputStream.read(DataInputStream.java:149)
>         at org.apache.hadoop.fs.FSInputChecker.readFully(FSInputChecker.java:436)
>         at org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.readChunk(ChecksumFileSystem.java:252)
>         at org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:276)
>         at org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:214)
>         at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:232)
>         at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:196)
>         - locked <0x00000007deecb978> (a org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker)
>         at java.io.DataInputStream.readFully(DataInputStream.java:195)
>         at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:120)
>         at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2359)
>         - locked <0x00000007deec8f70> (a org.apache.hadoop.io.SequenceFile$Reader)
>         at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2491)
>         - locked <0x00000007deec8f70> (a org.apache.hadoop.io.SequenceFile$Reader)
>         at org.apache.hadoop.mapred.SequenceFileRecordReader.next(SequenceFileRecordReader.java:82)
>         - locked <0x00000007deec82f0> (a org.apache.hadoop.mapred.SequenceFileRecordReader)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.nextBlock(RowContainer.java:360)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.next(RowContainer.java:267)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.next(RowContainer.java:74)
>         at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:644)
>         at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.checkAndGenObject(CommonJoinOperator.java:750)
>         at org.apache.hadoop.hive.ql.exec.JoinOperator.endGroup(JoinOperator.java:256)
>         at org.apache.hadoop.hive.ql.exec.mr.ExecReducer.close(ExecReducer.java:284)
>         at org.apache.hadoop.mapred.ReduceTask.runOldReducer(ReduceTask.java:453)
>         at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:392)
>         at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:163)
>         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:1671)
> {code}
> at 2016-07-15 07:15:35
> {code}
> "main" prio=10 tid=0x00007f90ec017000 nid=0xd193 runnable [0x00007f90f62e5000]
>    java.lang.Thread.State: RUNNABLE
>         at java.util.zip.CRC32.updateBytes(Native Method)
>         at java.util.zip.CRC32.update(CRC32.java:65)
>         at org.apache.hadoop.fs.FSInputChecker.verifySums(FSInputChecker.java:316)
>         at org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:279)
>         at org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:214)
>         at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:232)
>         at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:196)
>         - locked <0x00000007d68db510> (a org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker)
>         at java.io.DataInputStream.readFully(DataInputStream.java:195)
>         at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:120)
>         at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2359)
>         - locked <0x00000007d68d8b68> (a org.apache.hadoop.io.SequenceFile$Reader)
>         at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2491)
>         - locked <0x00000007d68d8b68> (a org.apache.hadoop.io.SequenceFile$Reader)
>         at org.apache.hadoop.mapred.SequenceFileRecordReader.next(SequenceFileRecordReader.java:82)
>         - locked <0x00000007d68d7f08> (a org.apache.hadoop.mapred.SequenceFileRecordReader)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.nextBlock(RowContainer.java:360)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.next(RowContainer.java:267)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.next(RowContainer.java:74)
>         at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:644)
>         at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.checkAndGenObject(CommonJoinOperator.java:750)
>         at org.apache.hadoop.hive.ql.exec.JoinOperator.endGroup(JoinOperator.java:256)
>         at org.apache.hadoop.hive.ql.exec.mr.ExecReducer.close(ExecReducer.java:284)
>         at org.apache.hadoop.mapred.ReduceTask.runOldReducer(ReduceTask.java:453)
>         at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:392)
>         at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:163)
>         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:1671)
>         at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
> {code}
> at 2016-07-15 07:19:10
> {code}
> "main" prio=10 tid=0x00007f90ec017000 nid=0xd193 runnable [0x00007f90f62e5000]
>    java.lang.Thread.State: RUNNABLE
>         at java.io.FileInputStream.readBytes(Native Method)
>         at java.io.FileInputStream.read(FileInputStream.java:272)
>         at org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileInputStream.read(RawLocalFileSystem.java:154)
>         at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
>         at java.io.BufferedInputStream.read1(BufferedInputStream.java:275)
>         at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
>         - locked <0x00000007df731218> (a org.apache.hadoop.fs.BufferedFSInputStream)
>         at java.io.DataInputStream.read(DataInputStream.java:149)
>         at org.apache.hadoop.fs.FSInputChecker.readFully(FSInputChecker.java:436)
>         at org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.readChunk(ChecksumFileSystem.java:252)
>         at org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:276)
>         at org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:214)
>         at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:232)
>         at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:196)
>         - locked <0x00000007df72dc20> (a org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker)
>         at java.io.DataInputStream.readFully(DataInputStream.java:195)
>         at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:120)
>         at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2359)
>         - locked <0x00000007df72b278> (a org.apache.hadoop.io.SequenceFile$Reader)
>         at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2491)
>         - locked <0x00000007df72b278> (a org.apache.hadoop.io.SequenceFile$Reader)
>         at org.apache.hadoop.mapred.SequenceFileRecordReader.next(SequenceFileRecordReader.java:82)
>         - locked <0x00000007df72a618> (a org.apache.hadoop.mapred.SequenceFileRecordReader)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.nextBlock(RowContainer.java:360)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.nextBlock(RowContainer.java:373)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.next(RowContainer.java:267)
>         at org.apache.hadoop.hive.ql.exec.persistence.RowContainer.next(RowContainer.java:74)
>         at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genUniqueJoinObject(CommonJoinOperator.java:644)
>         at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.checkAndGenObject(CommonJoinOperator.java:750)
>         at org.apache.hadoop.hive.ql.exec.JoinOperator.endGroup(JoinOperator.java:256)
>         at org.apache.hadoop.hive.ql.exec.mr.ExecReducer.close(ExecReducer.java:284)
>         at org.apache.hadoop.mapred.ReduceTask.runOldReducer(ReduceTask.java:453)
>         at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:392)
>         at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:163)
>         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:1671)
>         at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
> {code}
> You can see all three stack traces show CommonJoinOperator.genUniqueJoinObject was called by ExecReducer.close.



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