You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "Matthew Rocklin (JIRA)" <ji...@apache.org> on 2018/12/29 22:46:00 UTC

[jira] [Commented] (ARROW-1983) [Python] Add ability to write parquet `_metadata` file

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

Matthew Rocklin commented on ARROW-1983:
----------------------------------------

> If I understand correctly, we need to combine all of the row group metadata for all files in a directory.

Yes.  Ideally when writing a row group we would get some metadata object in memory. We would then collect all of those objects and hand them to some `write_metadata` function afterwards.

> When a new file is written, does this file have to be updated?
 
Yes, or it can be removed/invalidated.
 
As a side note, this is probably one of a small number of issues that stop Dask Dataframe from using PyArrow by default.  Metadata files with full row group information are especially valuable for us, particularly with remote/cloud storage.  (I'm going through Dask's parquet handling now)

> [Python] Add ability to write parquet `_metadata` file
> ------------------------------------------------------
>
>                 Key: ARROW-1983
>                 URL: https://issues.apache.org/jira/browse/ARROW-1983
>             Project: Apache Arrow
>          Issue Type: Improvement
>          Components: Python
>            Reporter: Jim Crist
>            Assignee: Robert Gruener
>            Priority: Major
>              Labels: beginner, parquet
>             Fix For: 0.13.0
>
>
> Currently {{pyarrow.parquet}} can only write the {{_common_metadata}} file (mostly just schema information). It would be useful to add the ability to write a {{_metadata}} file as well. This should include information about each row group in the dataset, including summary statistics. Having this summary file would allow filtering of row groups without needing to access each file beforehand.
> This would require that the user is able to get the written RowGroups out of a {{pyarrow.parquet.write_table}} call and then give these objects as a list to new function that then passes them on as C++ objects to {{parquet-cpp}} that generates the respective {{_metadata}} file.



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