[
https://issues.apache.org/jira/browse/PARQUET-1633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17360695#comment-17360695
]
ASF GitHub Bot commented on PARQUET-1633:
-----------------------------------------
gszadovszky commented on pull request #902:
URL: https://github.com/apache/parquet-mr/pull/902#issuecomment-858478591
@eadwright, I understand your concerns I don't really like it either.
Meanwhile, I don't feel good having a test that is not executed automatically.
Without regular executions there is no guarantee that this test would be
executed ever again and even if someone would execute it it might fail because
of the lack of maintenance.
What do you think about the following options? @shangxinli, I'm also curious
about your ideas.
* Execute this test separately with a maven profile. I am not sure if the CI
allows allocating such large memory but with Xmx options we might give a try
and create a separate check for this test only.
* Similar to the previous with the profile but not executing in the CI ever.
Instead, we add some comments to the release doc so this test will be executed
at least once per release.
* Configuring the CI profile to skip this test but have it in the normal
scenario meaning the devs will execute it locally. There are a couple of cons
though. There is no guarantee that devs executes all the tests including this
one. It also can cause issues if the dev doesn't have enough memory and don't
know that the test failure is not related to the current change.
--
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:
[email protected]
> 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)