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

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

It took me a while to understand the actual issue with partition columns not 
being included in the metadata, but so to get concrete, we currently have an 
example in the docs about how you can write a {{_metadata}} file using the 
metadata_collector and write_metadata utilities 
(https://arrow.apache.org/docs/python/parquet.html#writing-metadata-and-common-medata-files)

{code:python}
# Write a dataset and collect metadata information of all written files
metadata_collector = []
pq.write_to_dataset(table, root_path, metadata_collector=metadata_collector)

# Write the ``_common_metadata`` parquet file without row groups statistics
pq.write_metadata(table.schema, root_path / '_common_metadata')

# Write the ``_metadata`` parquet file with row groups statistics of all files
pq.write_metadata(
    table.schema, root_path / '_metadata',
    metadata_collector=metadata_collector
)
{code}

Now, this example doesn't actually use a partitioned dataset. And once you add 
a {{partition_cols}} argument to the {{write_to_dataset}} call, the above 
snippet will fail (so it's a not very useful example ..).

For example:

{code:python}
table = pa.table({'Month': [5, 5, 5, 5, 6, 6, 6, 6], 'Day': [1, 1, 2, 2, 1, 1, 
2, 2], 'Temp': range(8)})

metadata_collector = []
pq.write_to_dataset(table, root_path, partition_cols=["Month", "Day"], 
metadata_collector=metadata_collector)

# Write the ``_metadata`` parquet file with row groups statistics of all files
pq.write_metadata(
   table.schema, root_path / '_metadata',
   metadata_collector=metadata_collector
)
{code}

gives:

{code}
---------------------------------------------------------------------------
RuntimeError                              Traceback (most recent call last)
<ipython-input-18-f3324e50d348> in <module>
      6 
      7 # Write the ``_metadata`` parquet file with row groups statistics of 
all files
----> 8 pq.write_metadata(
      9     table.schema, root_path / '_metadata',
     10     metadata_collector=metadata_collector

~/scipy/repos/arrow/python/pyarrow/parquet.py in write_metadata(schema, where, 
metadata_collector, **kwargs)
   2082     if metadata_collector is not None:
   2083         # ParquetWriter doesn't expose the metadata until it's written. 
Write
-> 2084         # it and read it again.
   2085         metadata = read_metadata(where)
   2086         for m in metadata_collector:

~/scipy/repos/arrow/python/pyarrow/_parquet.pyx in 
pyarrow._parquet.FileMetaData.append_row_groups()

RuntimeError: AppendRowGroups requires equal schemas.
{code}

Because indeed the {{table.schema}} passed to {{write_metadata}} is not 
matching the schemas of the collected metadata (and thus those can't be 
appended)

> [C++] [Dataset] pyarrow.parquet.write_to_dataset does not send full schema to 
> metadata_collector
> ------------------------------------------------------------------------------------------------
>
>                 Key: ARROW-13269
>                 URL: https://issues.apache.org/jira/browse/ARROW-13269
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: C++
>    Affects Versions: 4.0.0
>            Reporter: Weston Pace
>            Priority: Major
>
> If there are partition columns specified then the writers will only write the 
> non-partition columns and thus they will not contain the fields used for the 
> partition.



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

Reply via email to