[ https://issues.apache.org/jira/browse/HIVE-12450?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15015022#comment-15015022 ]
Hive QA commented on HIVE-12450: -------------------------------- Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12773101/HIVE-12450.4.patch {color:green}SUCCESS:{color} +1 due to 2 test(s) being added or modified. {color:red}ERROR:{color} -1 due to 4 failed/errored test(s), 9869 tests executed *Failed tests:* {noformat} TestHWISessionManager - did not produce a TEST-*.xml file org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_authorization_uri_import org.apache.hive.jdbc.TestSSL.testSSLVersion org.apache.hive.jdbc.miniHS2.TestHs2Metrics.testMetrics {noformat} Test results: http://ec2-174-129-184-35.compute-1.amazonaws.com/jenkins/job/PreCommit-HIVE-TRUNK-Build/6077/testReport Console output: http://ec2-174-129-184-35.compute-1.amazonaws.com/jenkins/job/PreCommit-HIVE-TRUNK-Build/6077/console Test logs: http://ec2-174-129-184-35.compute-1.amazonaws.com/logs/PreCommit-HIVE-TRUNK-Build-6077/ Messages: {noformat} Executing org.apache.hive.ptest.execution.TestCheckPhase Executing org.apache.hive.ptest.execution.PrepPhase Executing org.apache.hive.ptest.execution.ExecutionPhase Executing org.apache.hive.ptest.execution.ReportingPhase Tests exited with: TestsFailedException: 4 tests failed {noformat} This message is automatically generated. ATTACHMENT ID: 12773101 - PreCommit-HIVE-TRUNK-Build > OrcFileMergeOperator does not use correct compression buffer size > ----------------------------------------------------------------- > > Key: HIVE-12450 > URL: https://issues.apache.org/jira/browse/HIVE-12450 > Project: Hive > Issue Type: Bug > Components: ORC > Affects Versions: 1.2.0, 1.3.0, 1.2.1, 2.0.0 > Reporter: Prasanth Jayachandran > Assignee: Prasanth Jayachandran > Priority: Critical > Attachments: HIVE-12450.1.patch, HIVE-12450.2.patch, > HIVE-12450.3.patch, HIVE-12450.4.patch, zlib-hang.png > > > OrcFileMergeOperator checks for compatibility before merging orc files. This > compatibility check include checking compression buffer size. But the output > file that is created does not honor the compression buffer size and always > defaults to 256KB. This will not be a problem when reading the orc file but > can create unwanted memory pressure because of wasted space within > compression buffer. > This issue also can make the merged file unreadable under certain cases. For > example, if the original compression buffer size is 8KB and if > hive.exec.orc.default.buffer.size is set to 4KB. The merge file operator will > use 4KB instead of actual 8KB which can result in hanging of ORC reader (more > specifically ZlibCodec will wait for more compression buffers). > {code:title=jstack output for hanging issue} > "main" prio=5 tid=0x00007fc073000000 nid=0x1703 runnable [0x0000700000218000] > java.lang.Thread.State: RUNNABLE > at java.util.zip.Inflater.inflateBytes(Native Method) > at java.util.zip.Inflater.inflate(Inflater.java:259) > - locked <0x00000007f5d5fdc8> (a java.util.zip.ZStreamRef) > at > org.apache.hadoop.hive.ql.io.orc.ZlibCodec.decompress(ZlibCodec.java:94) > at > org.apache.hadoop.hive.ql.io.orc.InStream$CompressedStream.readHeader(InStream.java:238) > at > org.apache.hadoop.hive.ql.io.orc.InStream$CompressedStream.read(InStream.java:262) > at java.io.InputStream.read(InputStream.java:101) > at > com.google.protobuf.CodedInputStream.refillBuffer(CodedInputStream.java:737) > at > com.google.protobuf.CodedInputStream.isAtEnd(CodedInputStream.java:701) > at > com.google.protobuf.CodedInputStream.readTag(CodedInputStream.java:99) > at > org.apache.hadoop.hive.ql.io.orc.OrcProto$StripeFooter.<init>(OrcProto.java:10661) > at > org.apache.hadoop.hive.ql.io.orc.OrcProto$StripeFooter.<init>(OrcProto.java:10625) > at > org.apache.hadoop.hive.ql.io.orc.OrcProto$StripeFooter$1.parsePartialFrom(OrcProto.java:10730) > at > org.apache.hadoop.hive.ql.io.orc.OrcProto$StripeFooter$1.parsePartialFrom(OrcProto.java:10725) > at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:89) > at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:95) > at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:49) > at > org.apache.hadoop.hive.ql.io.orc.OrcProto$StripeFooter.parseFrom(OrcProto.java:10958) > at > org.apache.hadoop.hive.ql.io.orc.MetadataReaderImpl.readStripeFooter(MetadataReaderImpl.java:114) > at > org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.readStripeFooter(RecordReaderImpl.java:240) > at > org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.beginReadStripe(RecordReaderImpl.java:847) > at > org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.readStripe(RecordReaderImpl.java:818) > at > org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.advanceStripe(RecordReaderImpl.java:1033) > at > org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.advanceToNextRow(RecordReaderImpl.java:1068) > at > org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.<init>(RecordReaderImpl.java:217) > at > org.apache.hadoop.hive.ql.io.orc.ReaderImpl.rowsOptions(ReaderImpl.java:638) > at org.apache.hadoop.hive.ql.io.orc.ReaderImpl.rows(ReaderImpl.java:625) > at > org.apache.hadoop.hive.ql.io.orc.FileDump.printMetaData(FileDump.java:162) > at org.apache.hadoop.hive.ql.io.orc.FileDump.main(FileDump.java:110) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:606) > at org.apache.hadoop.util.RunJar.run(RunJar.java:221) > at org.apache.hadoop.util.RunJar.main(RunJar.java:136) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)