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

ASF GitHub Bot commented on PARQUET-1633:
-----------------------------------------

advancedxy commented on pull request #902:
URL: https://github.com/apache/parquet-mr/pull/902#issuecomment-841124415


   > @eadwright,
   > 
   > I'll try to summarize the issue, please correct me if I'm wrong. 
Parquet-mr is not able to write such big row groups (>2GB) because of the `int` 
array size limitation. Meanwhile, both the format and some other 
implementations allow such big row groups. So, parquet-mr shall be prepared for 
this issue in some way.
   > One option is to "simply" read the large row groups. It would require 
significant efforts to use proper memory handling objects that would properly 
support reading the large row groups. (A similar effort would also make 
parquet-mr available to write larger row groups than 2GB.)
   > 
   > The other option is to handle the too large row groups with a proper error 
message in parquet-mr without allowing silent overflows. This second option 
would be this effort. It is great to handle to potential int overflows but the 
main point, I think, would be at the footer conversion 
(`ParquetMetadataConverter`) where we create our own object structure from the 
file footer. At this point we can throw the proper error messages if the row 
group is too large to be handled (for now) in parquet-mr.
   > BTW, it might not be enough to check the potential overflows to validate 
if we can read a row group size. (See e.g. the source code of 
[ArrayList](https://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/java/util/ArrayList.java#l229).)
   > 
   > About the lack of the unit tests. I can accept in some cases where unit 
tests are not practically feasible to be implemented. In these cases I usually 
ask to validate the code offline.
   
   Hi @gszadovszky parquet-mr is able to produce big row groups. We found some 
files wrote by Spark(which uses parquet-mr) have this problem. See 
https://issues.apache.org/jira/browse/PARQUET-2045 for details.
   
   There are two options to fix this problem:
   1. fail at writer side when creating such large row group/column chunk
   2. support at reader side, which is this approach. It would require a lot of 
resource, but it's feasible.
   
   Either option is fine for me, WDYT?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Integer overflow in ParquetFileReader.ConsecutiveChunkList
> ----------------------------------------------------------
>
>                 Key: PARQUET-1633
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1633
>             Project: Parquet
>          Issue Type: Bug
>          Components: parquet-mr
>    Affects Versions: 1.10.1
>            Reporter: Ivan Sadikov
>            Priority: Major
>
> When reading a large Parquet file (2.8GB), I encounter the following 
> exception:
> {code:java}
> Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value 
> at 0 in block -1 in file 
> dbfs:/user/hive/warehouse/demo.db/test_table/part-00014-tid-1888470069989036737-593c82a4-528b-4975-8de0-5bcbc5e9827d-10856-1-c000.snappy.parquet
> at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
> at 
> org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
> at 
> org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:40)
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1$$anon$2.getNext(FileScanRDD.scala:228)
> ... 14 more
> Caused by: java.lang.IllegalArgumentException: Illegal Capacity: -212
> at java.util.ArrayList.<init>(ArrayList.java:157)
> at 
> org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:1169){code}
>  
> The file metadata is:
>  * block 1 (3 columns)
>  ** rowCount: 110,100
>  ** totalByteSize: 348,492,072
>  ** compressedSize: 165,689,649
>  * block 2 (3 columns)
>  ** rowCount: 90,054
>  ** totalByteSize: 3,243,165,541
>  ** compressedSize: 2,509,579,966
>  * block 3 (3 columns)
>  ** rowCount: 105,119
>  ** totalByteSize: 350,901,693
>  ** compressedSize: 144,952,177
>  * block 4 (3 columns)
>  ** rowCount: 48,741
>  ** totalByteSize: 1,275,995
>  ** compressedSize: 914,205
> I don't have the code to reproduce the issue, unfortunately; however, I 
> looked at the code and it seems that integer {{length}} field in 
> ConsecutiveChunkList overflows, which results in negative capacity for array 
> list in {{readAll}} method:
> {code:java}
> int fullAllocations = length / options.getMaxAllocationSize();
> int lastAllocationSize = length % options.getMaxAllocationSize();
>       
> int numAllocations = fullAllocations + (lastAllocationSize > 0 ? 1 : 0);
> List<ByteBuffer> buffers = new ArrayList<>(numAllocations);{code}
>  
> This is caused by cast to integer in {{readNextRowGroup}} method in 
> ParquetFileReader:
> {code:java}
> currentChunks.addChunk(new ChunkDescriptor(columnDescriptor, mc, startingPos, 
> (int)mc.getTotalSize()));
> {code}
> which overflows when total size of the column is larger than 
> Integer.MAX_VALUE.
> I would appreciate if you could help addressing the issue. Thanks!
>  



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

Reply via email to