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

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

bq.  I suppose the same question holds for common_metadata.  Should 
common_metadata contain only the columns written?  Or should it also contain 
the partition columns? 

The _common_metadata file typically doesn't hold any information about file (no 
row group info) but just the schema, so the intent here is to reflect the 
schema of the "full" dataset (so including partitioning columns). 

At least, that's what dask does. Continuing the example from above (reading the 
file written by dask):

{code:python}
>>> meta = pq.read_metadata(root_path2 / "_common_metadata")
>>> meta
<pyarrow._parquet.FileMetaData object at 0x7f1449286720>
  created_by: parquet-cpp-arrow version 5.0.0-SNAPSHOT
  num_columns: 4
  num_rows: 0
  num_row_groups: 0
  format_version: 1.0
  serialized_size: 2433

>>> meta.schema
<pyarrow._parquet.ParquetSchema object at 0x7f14368d5c80>
required group field_id=-1 schema {
  optional int64 field_id=-1 Month;
  optional int64 field_id=-1 Day;
  optional int64 field_id=-1 Temp;
  optional int64 field_id=-1 __null_dask_index__;
}
{code}

And to be clear, all this {{_metadata}} and {{_common_metadata}} is 
"implementation-defined", there is not a standard description or specification 
about what this is expected to do, there are just some implementations that use 
such files (spark in the past, dask still right now).  
(some discussion about this in eg ARROW-1983)

bq. Also, I think we need to support individual files having different 
metadata?  That should probably be tested as part of this feature as well.

Currently, if you combine multiple Parquet FileMetadata objects into a single 
one (appending the row groups), we require matching schemas. There was some 
discussion about that when this restriction was introduced (the change was 
introduced in https://github.com/apache/arrow/pull/7180 (ARROW-8062), some 
discussion of it on the dask side: https://github.com/dask/dask/issues/6243). 

Personally I think it could be fine to have the restriction that if you want to 
use the {{_metadata}} approach you can't have schema evolution (there are other 
"data lake management" tools that are more advanced than a metadata file).

> [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