You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@orc.apache.org by "Stamatis Zampetakis (Jira)" <ji...@apache.org> on 2023/02/08 14:13:00 UTC

[jira] [Commented] (ORC-1361) InvalidProtocolBufferException when reading large stripe statistics

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

Stamatis Zampetakis commented on ORC-1361:
------------------------------------------

[~deshanxiao] After more testing and thinking it turns out that reading everything into byte[], ByteBuffer, etc., cannot work and it is not a good idea either.

After writing a file with stripe statistics greater than 2GB and setting the protobuf size limit to Integer.MAX_VALUE (2GB) I get consistently the following exception. No matter if I use my naive implementation ({{parseMetadataFromByteArray}}) or the existing one ({{InStream.createCodedInputStream}}) with increased limit.
{noformat}
com.google.protobuf.InvalidProtocolBufferException: While parsing a protocol message, the input ended unexpectedly in the middle of a field.  This could mean either that the input has been truncated or that an embedded message misreported its own length.
	at com.google.protobuf.InvalidProtocolBufferException.truncatedMessage(InvalidProtocolBufferException.java:107)
	at com.google.protobuf.CodedInputStream$StreamDecoder.pushLimit(CodedInputStream.java:2729)
	at com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2421)
	at org.apache.orc.OrcProto$StripeStatistics.<init>(OrcProto.java:23549)
	at org.apache.orc.OrcProto$StripeStatistics.<init>(OrcProto.java:23499)
	at org.apache.orc.OrcProto$StripeStatistics$1.parsePartialFrom(OrcProto.java:24247)
	at org.apache.orc.OrcProto$StripeStatistics$1.parsePartialFrom(OrcProto.java:24241)
	at com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2423)
	at org.apache.orc.OrcProto$Metadata.<init>(OrcProto.java:24352)
	at org.apache.orc.OrcProto$Metadata.<init>(OrcProto.java:24302)
	at org.apache.orc.OrcProto$Metadata$1.parsePartialFrom(OrcProto.java:25048)
	at org.apache.orc.OrcProto$Metadata$1.parsePartialFrom(OrcProto.java:25042)
	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 com.google.protobuf.GeneratedMessageV3.parseWithIOException(GeneratedMessageV3.java:357)
	at org.apache.orc.OrcProto$Metadata.parseFrom(OrcProto.java:24557)
	at org.apache.orc.impl.ReaderImpl.parseMetadataFromByteArray(ReaderImpl.java:1056)
	at org.apache.orc.impl.ReaderImpl.deserializeStripeStats(ReaderImpl.java:1041)
	at org.apache.orc.impl.ReaderImpl.getVariantStripeStatistics(ReaderImpl.java:326)
	at org.apache.orc.impl.ReaderImpl.getStripeStatistics(ReaderImpl.java:1089)
	at org.apache.orc.impl.ReaderImpl.getStripeStatistics(ReaderImpl.java:1076)
	at org.apache.orc.TestOrcWithLargeStripeStatistics.testGetStripeStatistics(TestOrcWithLargeStripeStatistics.java:41)
{noformat}

I also logged https://github.com/protocolbuffers/protobuf/issues/11729 since the problem of large protobuf messages is not something specific to ORC but affects protobuf in general.

I think that the best solution at this point is to catch the exception coming from protobuf and return an empty list as I did in https://github.com/apache/orc/pull/1402.

Increasing the protobuf limit further doesn't seem like a very good idea since more problems may occur due to the increased memory consumption (e.g., OutOfMemoryError, OOM Killer, GC pauces, etc.). Most importantly, the increased limit (to the max 2GB) is not sufficient to solve the problem for good since we can have stripe statistics that exceed the 2GB as well as shown above.

For the future, I think we should start a broader discussion and re-evaluate the design and usage of protobuf messages for ORC. For instance, in certain places we should be better of using flat instead of nested (struct) messages.

> InvalidProtocolBufferException when reading large stripe statistics
> -------------------------------------------------------------------
>
>                 Key: ORC-1361
>                 URL: https://issues.apache.org/jira/browse/ORC-1361
>             Project: ORC
>          Issue Type: Bug
>          Components: Java
>    Affects Versions: 1.8.2
>            Reporter: Stamatis Zampetakis
>            Priority: Major
>         Attachments: TestOrcWithLargeStripeStatistics.java
>
>
> Any attempt to obtain the stripe statistics from an ORC file with a metadata section exceeding the hardcoded protobuf 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}
> 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:154)
> 	at com.google.protobuf.CodedInputStream$StreamDecoder.readRawBytesSlowPathOneChunk(CodedInputStream.java:2954)
> 	at com.google.protobuf.CodedInputStream$StreamDecoder.readBytesSlowPath(CodedInputStream.java:3035)
> 	at com.google.protobuf.CodedInputStream$StreamDecoder.readBytes(CodedInputStream.java:2446)
> 	at org.apache.orc.OrcProto$StringStatistics.<init>(OrcProto.java:2118)
> 	at org.apache.orc.OrcProto$StringStatistics.<init>(OrcProto.java:2070)
> 	at org.apache.orc.OrcProto$StringStatistics$1.parsePartialFrom(OrcProto.java:3285)
> 	at org.apache.orc.OrcProto$StringStatistics$1.parsePartialFrom(OrcProto.java:3279)
> 	at com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2423)
> 	at org.apache.orc.OrcProto$ColumnStatistics.<init>(OrcProto.java:8172)
> 	at org.apache.orc.OrcProto$ColumnStatistics.<init>(OrcProto.java:8093)
> 	at org.apache.orc.OrcProto$ColumnStatistics$1.parsePartialFrom(OrcProto.java:10494)
> 	at org.apache.orc.OrcProto$ColumnStatistics$1.parsePartialFrom(OrcProto.java:10488)
> 	at com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2423)
> 	at org.apache.orc.OrcProto$StripeStatistics.<init>(OrcProto.java:23549)
> 	at org.apache.orc.OrcProto$StripeStatistics.<init>(OrcProto.java:23499)
> 	at org.apache.orc.OrcProto$StripeStatistics$1.parsePartialFrom(OrcProto.java:24247)
> 	at org.apache.orc.OrcProto$StripeStatistics$1.parsePartialFrom(OrcProto.java:24241)
> 	at com.google.protobuf.CodedInputStream$StreamDecoder.readMessage(CodedInputStream.java:2423)
> 	at org.apache.orc.OrcProto$Metadata.<init>(OrcProto.java:24352)
> 	at org.apache.orc.OrcProto$Metadata.<init>(OrcProto.java:24302)
> 	at org.apache.orc.OrcProto$Metadata$1.parsePartialFrom(OrcProto.java:25048)
> 	at org.apache.orc.OrcProto$Metadata$1.parsePartialFrom(OrcProto.java:25042)
> 	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 com.google.protobuf.GeneratedMessageV3.parseWithIOException(GeneratedMessageV3.java:357)
> 	at org.apache.orc.OrcProto$Metadata.parseFrom(OrcProto.java:24557)
> 	at org.apache.orc.impl.ReaderImpl.deserializeStripeStats(ReaderImpl.java:1040)
> 	at org.apache.orc.impl.ReaderImpl.getVariantStripeStatistics(ReaderImpl.java:325)
> 	at org.apache.orc.impl.ReaderImpl.getStripeStatistics(ReaderImpl.java:1074)
> 	at org.apache.orc.impl.ReaderImpl.getStripeStatistics(ReaderImpl.java:1061)
> {noformat}
> There are various ways of ending up with an ORC file that has a large metadata section since the write never fails. 
> Once the file is created it is no longer possible to read back all the information correctly.
> In versions without ORC-520 (before 1.6.0) the file cannot be read at all since stripe statistics are read eagerly in the constructor of the ReaderImpl.
> In versions with ORC-520 (1.6.0 onwards) the exception is raised only when trying to read explicitly the stripe statistics.
> Attached a test case (TestOrcWithLargeStripeStatistics.java) reproducing the problem in current main branch (2ff9001ddef082eaa30e21cbb034f266e0721664).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)