[
https://issues.apache.org/jira/browse/ARROW-8655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17375477#comment-17375477
]
Joris Van den Bossche commented on ARROW-8655:
----------------------------------------------
Looking at what information is available in the legacy ParquetDataset:
{code:python}
import pandas as pd
import pyarrow as pa
import pyarrow.parquet as pq
import pyarrow.dataset as ds
df = pd.DataFrame({"year": [2020, 2020, 2021, 2021], "month":[1, 2, 1, 2],
"values": [1, 2, 3, 4]})
df.to_parquet("test_partitioned", partition_cols=["year", "month"],
engine="pyarrow")
{code}
{code:python}
In [2]: d1 = pq.ParquetDataset("test_partitioned/")
In [4]: piece = d1.pieces[0]
# a single piece has information in "partition_keys"
In [5]: piece
Out[5]:
ParquetDatasetPiece('test_partitioned//year=2020/month=1/1b27b290ffdf4cccbf21d2d2feef79f4.parquet',
row_group=None, partition_keys=[('year', 0), ('month', 0)])
# note: the values are the *indices* into the unique values per partition
field, not the actual value
In [6]: piece.partition_keys
Out[6]: [('year', 0), ('month', 0)]
# In addition, the dataset object has a "partitions" attribute
In [7]: d1.partitions
Out[7]: <pyarrow.parquet.ParquetPartitions at 0x7fe4caea22e0>
# the partition field names (note: also not ordered here)
In [8]: d1.partitions.partition_names
Out[8]: {'month', 'year'}
# more information about each partition field
In [9]: d1.partitions.levels
Out[9]:
[<pyarrow.parquet.PartitionSet at 0x7fe4c9ea8760>,
<pyarrow.parquet.PartitionSet at 0x7fe4cae94e20>]
In [10]: d1.partitions.levels[0].dictionary
Out[10]:
<pyarrow.lib.Int64Array object at 0x7fe481dfafa0>
[
2020,
2021
]
In [11]: d1.partitions.levels[0].keys
Out[11]: ['2020', '2021']
In [12]: d1.partitions.levels[0].key_indices
Out[12]: {'2020': 0, '2021': 1}
In [13]: d1.partitions.levels[0].name
Out[13]: 'year'
{code}
I am working on a draft PR to preserve the {{Partitioning}} object from which
the {{FileSystemDataset}} was created in the new API. This could look like:
{code:python}
In [16]: d2 = ds.dataset("test_partitioned/", partitioning="hive")
In [17]: d2
Out[17]: <pyarrow._dataset.FileSystemDataset at 0x7fe481b6e270>
In [18]: d2.partitioning
Out[18]: <pyarrow._dataset.HivePartitioning at 0x7fe48162c7f0>
In [19]: d2.partitioning.schema
Out[19]:
year: int32
month: int32
In [20]: d2.partitioning.schema.names
Out[20]: ['year', 'month']
In [21]: d2.partitioning.dictionaries
Out[21]:
[<pyarrow.lib.Int32Array object at 0x7fe480fd9fa0>
[
2020,
2021
],
<pyarrow.lib.Int32Array object at 0x7fe480fd9b80>
[
1,
2
]]
{code}
I think this should provide all necessary information?
In addition, in the new API, we have the {{Fragments}}, which currently already
allow the following:
{code:python}
In [22]: fragments = list(d2.get_fragments())
In [23]: fragments[0].partition_expression
Out[23]: <pyarrow.dataset.Expression ((year == 2020) and (month == 1))>
In [24]: ds._get_partition_keys(fragments[0].partition_expression)
Out[24]: {'month': 1, 'year': 2020}
{code}
We could add a public property on the {{Fragment}} to expose this, for example
a {{FileFragment.partition_values}} (or other name if there are suggestions).
Feedback on whether this would be useful / sufficient for dask (or other use
cases) would be very welcome!
> [C++][Dataset][Python][R] Preserve partitioning information for a discovered
> Dataset
> ------------------------------------------------------------------------------------
>
> Key: ARROW-8655
> URL: https://issues.apache.org/jira/browse/ARROW-8655
> Project: Apache Arrow
> Issue Type: Improvement
> Components: C++
> Reporter: Joris Van den Bossche
> Priority: Major
> Labels: dataset, dataset-dask-integration, pull-request-available
> Fix For: 6.0.0
>
> Time Spent: 1h 20m
> Remaining Estimate: 0h
>
> Currently, we have the {{HivePartitioning}} and {{DirectoryPartitioning}}
> classes that describe a partitioning used in the discovery phase. But once a
> dataset object is created, it doesn't know any more about this, it just has
> partition expressions for the fragments. And the partition keys are added to
> the schema, but you can't directly know which columns of the schema
> originated from the partitions.
> However, there can be use cases where it would be useful that a dataset still
> "knows" from what kind of partitioning it was created:
> - The "read CSV write back Parquet" use case, where the CSV was already
> partitioned and you want to automatically preserve that partitioning for
> parquet (kind of roundtripping the partitioning on read/write)
> - To convert the dataset to other representation, eg conversion to pandas, it
> can be useful to know what columns were partition columns (eg for pandas,
> those columns might be good candidates to be set as the index of the
> pandas/dask DataFrame). I can imagine conversions to other systems can use
> similar information.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)