Re: Metadata for partitioned datasets in pyarrow.parquet

2019-05-21 Thread Richard Zamora
Thank you for the responses Wes and Joris! These summaries are very helpful to me. I decided to look into ARROW-5349 to get my feet wet, and just submitted a WIP PR (https://github.com/apache/arrow/pull/4361). If you get a chance, please take a look and provide feedback. I have limited exper

Re: Metadata for partitioned datasets in pyarrow.parquet

2019-05-16 Thread Joris Van den Bossche
Missed the email of Wes, but yeah, I think we basically said the same. Answer to another question you raised in the notebook: > [about writing a _common_metadata file] ... uses the schema object for > the 0th partition. This actually means that not *all* information in > _common_metadata will be

Re: Metadata for partitioned datasets in pyarrow.parquet

2019-05-16 Thread Joris Van den Bossche
Hi Rick, Thanks for exploring this! I am still quite new to Parquet myself, so the following might not be fully correct, but based on my current understanding, to enable projects like dask to write the different pieces of a Parquet dataset using pyarrow, we need the following functionalities: -

Re: Metadata for partitioned datasets in pyarrow.parquet

2019-05-16 Thread Wes McKinney
hi Richard, We have been discussing this in https://issues.apache.org/jira/browse/ARROW-1983 All that is currently missing is (AFAICT): * A C++ function to write a vector of FileMetaData as a _metadata file (make sure the file path is set in the metadata objects) * A Python binding for this Th