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

Joris Van den Bossche commented on ARROW-10131:
-----------------------------------------------

One idea could be to not yet parse statistics in {{ParquetDatasetFactory}}, and 
to provide a columns keyword in {{EnsureCompleteMetadata}} (which could then be 
specified eg when filtering row groups to only ensure the metadata of columns 
involved in the filter). However, to avoid reading the metadata / footers again 
when doing that, we would need to store the parquet FileMetadata / 
RowGroupMetadata object in the fragments. 

So _if_ storing the FileMetadata / RowGroupMetadata object in the fragments, we 
could probably go for a fully "lazy" access as well: {{EnsureCompleteMetadata}} 
could only ensure that the underlying parquet metadata object is stored, but 
the actual statistics get only parsed when actually accessing the RowGroupInfo 
statistics. 
However, currently the statistics are stored on the {{RowGroupInfo}} as a 
{{StructScalar}}, so this would also need a refactor if we want to allow 
accessing/parsing only the statistics of a certain column.

I also don't fully know how feasable it is to actually store the FileMetadData 
/ RowGroupMetadata on the fragment, because when eg splitting it in a fragment 
per row group, we would either need to copy the FileMetaData object multiple 
times, or the different fragments would share the same metadata.



> [C++][Dataset] Lazily parse parquet metadata / statistics in 
> ParquetDatasetFactory and ParquetFileFragment
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: ARROW-10131
>                 URL: https://issues.apache.org/jira/browse/ARROW-10131
>             Project: Apache Arrow
>          Issue Type: Improvement
>          Components: C++
>            Reporter: Joris Van den Bossche
>            Priority: Major
>              Labels: dataset, dataset-dask-integration
>
> Related to ARROW-9730, parsing of the statistics in parquet metadata is 
> expensive, and therefore should be avoided when possible.
> For example, the {{ParquetDatasetFactory}} ({{ds.parquet_dataset()}} in 
> python) parses all statistics of all files and all columns. While when doing 
> a filtered read, you might only need the statistics of certain files (eg if a 
> filter on a partition field already excluded many files) and certain columns 
> (eg only the columns on which you are actually filtering).
> The current API is a bit all-or-nothing (both ParquetDatasetFactory, or a 
> later EnsureCompleteMetadata parse all statistics, and don't allow parsing a 
> subset, or only parsing the other (non-statistics) metadata, ...), so I think 
> we should try to think of better abstractions.
> cc [~rjzamora] [~bkietz]



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

Reply via email to