Ian O Connell created SPARK-24230:
-------------------------------------

             Summary: With Parquet 1.10 upgrade has errors in the vectorized 
reader
                 Key: SPARK-24230
                 URL: https://issues.apache.org/jira/browse/SPARK-24230
             Project: Spark
          Issue Type: Bug
          Components: SQL
    Affects Versions: 2.4.0
            Reporter: Ian O Connell


When reading some parquet files can get an error like:

java.io.IOException: expecting more rows but reached last block. Read 0 out of 
1194236

This happens when looking for a needle thats pretty rare in a large haystack.

 

The issue here I believe is that the total row count is calculated at

[https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java#L229]

 

But we pass the blocks we filtered via 

org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups

to the ParquetFileReader constructor.

 

However the ParquetFileReader constructor will filter the list of blocks again 
using

 

[https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L737]

 

if a block is filtered out by the latter method, and not the former the 
vectorized reader will believe it should see more rows than it will.

the fix I used locally is pretty straight forward:
{code:java}
for (BlockMetaData block : blocks) {
this.totalRowCount += block.getRowCount();
}
{code}
goes to
{code:java}
this.totalRowCount = this.reader.getRecordCount();
{code}
[~rdblue] do you know if this sounds right? The second filter method in the 
ParquetFileReader might filter more blocks leading to the count being off? 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to