[
https://issues.apache.org/jira/browse/ARROW-1983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16817308#comment-16817308
]
Pearu Peterson commented on ARROW-1983:
---------------------------------------
Currently, ParquetDataset metadata has the following approximate data structure
(type-specs are shown only for the relevant attributes):
{noformat}
ParquetDataset:
list<ParquetDatasetPiece> pieces
list<str> paths
fs
common_metadata, common_metadata_path
metadata, metadata_path
ParquetDatasetPiece:
sting path
get_metadata() -> FileMetaData
partition_keys
FileMetaData:
list<RowGroupMetaData> row_groups
ParquetSchema schema
dict metadata = {b‘pandas’: <pandas metadata in bytes>}
int num_rows, num_columns
str format_version, created_by
RowGroupMetaData:
list<ColumnChunkMetaData> columns
int num_rows, total_byte_size
ColumnChunkMetaData:
str physical_type, encodings, path_in_schema, compression
int num_values, total_uncompressed_size, total_compressed_size,
data_page_offset, index_page_offset, dictionary_page_offset
RowGroupStatistics statistics
RowGroupStatistics:
bool has_min_max
int min, max, null_count, distinct_count, num_values
str physical_type{noformat}
If only the data in RowGroupStatistics is relevant for this issue (please
confirm), then the statistics data could be collected into a single Parquet
file, say `_statistics`, containing the following columns:
{noformat}
<Piece.path>, <RowGroup index>, <ColumnChunk index>, <RowGroupStatistics
data>{noformat}
[~mrocklin], would the information in `_statistics` sufficient for Dask needs?
> [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: C++, Python
> Reporter: Jim Crist
> Priority: Major
> Labels: beginner, parquet
> Fix For: 0.14.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)