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

Weston Pace commented on ARROW-8655:
------------------------------------

I think the only thing tricky here is the unique values.  In the general case a 
dataset may not know all possible values.  The "partition_expression" of a 
fragment is not required to be an equality expression (or even several ANDed 
together).  Technically there is nothing against creating a union dataset, 
perhaps composed of a CSV dataset (where all data has timestamp < 2020) and a 
parquet dataset (where all data has timestamp > 2020) because the company 
changed how they stored data at some point.

Scanning for all values currently used is something that happens in the 
factory->dataset part (which I suppose is kind of hidden in the current python 
implementation).  Maybe there is something we can add to the dataset factory so 
that calling Finish (or perhaps adding a new property that can be accessed 
after calling Finish) could return dictionaries of everything it discovered.

> [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: 5.0.0
>
>          Time Spent: 1h 50m
>  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)

Reply via email to