[ https://issues.apache.org/jira/browse/HIVE-26987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878852#comment-17878852 ]
Stamatis Zampetakis commented on HIVE-26987: -------------------------------------------- [~zhangbutao] We need to attempt to run the repro again and check if it passes in order to mark this as resolved. With ORC-1361 we will get an empty list instead of an exception but we may need to add some special handling for the empty list in Hive. > InvalidProtocolBufferException when reading column statistics from ORC files > ---------------------------------------------------------------------------- > > Key: HIVE-26987 > URL: https://issues.apache.org/jira/browse/HIVE-26987 > Project: Hive > Issue Type: Bug > Components: HiveServer2, ORC > Affects Versions: 3.1.0, 4.0.0-alpha-2 > Reporter: Stamatis Zampetakis > Priority: Major > Attachments: data.csv.gz, orc_large_column_metadata.q > > > Any attempt to read an ORC file (query an ORC table) having a metadata > section with column statistics exceeding the hardcoded limit of 1GB > ([https://github.com/apache/orc/blob/2ff9001ddef082eaa30e21cbb034f266e0721664/java/core/src/java/org/apache/orc/impl/InStream.java#L41]) > leads to the following exception. > {noformat} > Caused by: com.google.protobuf.InvalidProtocolBufferException: Protocol > message was too large. May be malicious. Use > CodedInputStream.setSizeLimit() to increase the size limit. > at > com.google.protobuf.InvalidProtocolBufferException.sizeLimitExceeded(InvalidProtocolBufferException.java:162) > at > com.google.protobuf.CodedInputStream$StreamDecoder.readRawBytesSlowPathOneChunk(CodedInputStream.java:2940) > at > com.google.protobuf.CodedInputStream$StreamDecoder.readBytesSlowPath(CodedInputStream.java:3021) > at > com.google.protobuf.CodedInputStream$StreamDecoder.readBytes(CodedInputStream.java:2432) > at org.apache.orc.OrcProto$StringStatistics.<init>(OrcProto.java:1718) > at org.apache.orc.OrcProto$StringStatistics.<init>(OrcProto.java:1663) > at > org.apache.orc.OrcProto$StringStatistics$1.parsePartialFrom(OrcProto.java:1766) > at > org.apache.orc.OrcProto$StringStatistics$1.parsePartialFrom(OrcProto.java:1761) > at > com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2409) > at org.apache.orc.OrcProto$ColumnStatistics.<init>(OrcProto.java:6552) > at org.apache.orc.OrcProto$ColumnStatistics.<init>(OrcProto.java:6468) > at > org.apache.orc.OrcProto$ColumnStatistics$1.parsePartialFrom(OrcProto.java:6678) > at > org.apache.orc.OrcProto$ColumnStatistics$1.parsePartialFrom(OrcProto.java:6673) > at > com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2409) > at > org.apache.orc.OrcProto$StripeStatistics.<init>(OrcProto.java:19586) > at > org.apache.orc.OrcProto$StripeStatistics.<init>(OrcProto.java:19533) > at > org.apache.orc.OrcProto$StripeStatistics$1.parsePartialFrom(OrcProto.java:19622) > at > org.apache.orc.OrcProto$StripeStatistics$1.parsePartialFrom(OrcProto.java:19617) > at > com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2409) > at org.apache.orc.OrcProto$Metadata.<init>(OrcProto.java:20270) > at org.apache.orc.OrcProto$Metadata.<init>(OrcProto.java:20217) > at > org.apache.orc.OrcProto$Metadata$1.parsePartialFrom(OrcProto.java:20306) > at > org.apache.orc.OrcProto$Metadata$1.parsePartialFrom(OrcProto.java:20301) > at > com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:86) > at > com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:91) > at > com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:48) > at org.apache.orc.OrcProto$Metadata.parseFrom(OrcProto.java:20438) > at > org.apache.orc.impl.ReaderImpl.deserializeStripeStats(ReaderImpl.java:1013) > at > org.apache.orc.impl.ReaderImpl.getVariantStripeStatistics(ReaderImpl.java:317) > at > org.apache.orc.impl.ReaderImpl.getStripeStatistics(ReaderImpl.java:1047) > at > org.apache.orc.impl.ReaderImpl.getStripeStatistics(ReaderImpl.java:1034) > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator.populateAndCacheStripeDetails(OrcInputFormat.java:1679) > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator.callInternal(OrcInputFormat.java:1557) > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator.access$2900(OrcInputFormat.java:1342) > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator$1.run(OrcInputFormat.java:1529) > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator$1.run(OrcInputFormat.java:1526) > 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:1878) > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator.call(OrcInputFormat.java:1526) > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator.call(OrcInputFormat.java:1342) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > {noformat} > There are various ways of creating such a file and once this happens it is no > longer possible to read it back. A complete reproducer of the problem using > Hive is attached in orc_large_column_metadata.q file. > Reproducible in current master (2031af314e70f3b8e07add13cb65416c29956181) by > running: > {code:bash} > mvn test -Dtest=TestMiniLlapLocalCliDriver -Dqfile=orc_large_column_metadata.q > {code} > Increase java heap accordingly "-Xmx8g" while running the test to avoid > hitting OOM before the actual error. -- This message was sent by Atlassian Jira (v8.20.10#820010)