You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@arrow.apache.org by "Joris Van den Bossche (Jira)" <ji...@apache.org> on 2021/09/01 08:08:00 UTC

[jira] [Commented] (ARROW-13813) [C++][Dataset] Support URL encoding of partition field values for the file path

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

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

Currently we don't support "transforms" for the partitioning column (something we maybe should? so that you could also say "year(date_column)" to partition on), which means that you need to calculate such URL encoded column up front, which is not necessarily ideal, both performance/memory wise (although in a (lazy/batched) query execution context, this might not matter) and from a usability context.

(especially for that last point (usability), I think it would be nice to support this within the "write to partitioned data" step)

Another thought: currently, when you partition on a column with string values, you can easily and silently get invalid directories. Small code snippet:

{code:python}
import pyarrow.dataset as ds

table = pa.table({'a': ['A', 'B', 'A/B', ''], 'b': range(4)})
ds.write_dataset(table, "test_dataset_invalid_strings.parquet", format="parquet", partitioning=ds.partitioning(table.select(['a']).schema))
ds.dataset("test_dataset_invalid_strings.parquet/", partitioning=['a']).to_table().to_pandas()
{code}

Not necessarily directly this issue, but related question: should we write this silently? Or should we actually check when creating the file path that the value inserted for the partition field is a "valid" string for file paths (so eg no /, not an empty string, ..), and raise an error instead of creating a wrong dataset? Or should we automatically encode those? 
(I don't know what the overhead would be of validating those strings, but it would also only be needed for certain data types, eg not if your partition field was originally integer)



> [C++][Dataset] Support URL encoding of partition field values for the file path
> -------------------------------------------------------------------------------
>
>                 Key: ARROW-13813
>                 URL: https://issues.apache.org/jira/browse/ARROW-13813
>             Project: Apache Arrow
>          Issue Type: Improvement
>          Components: C++
>            Reporter: Joris Van den Bossche
>            Priority: Major
>              Labels: dataset
>
> In ARROW-12644, we added support for _decoding_ the file paths when reading datasets. So a valid follow-up question: should we also support _encoding_ when writing datasets?
> (see also https://github.com/apache/arrow/issues/11027)
> Rereading ARROW-12644, there wasn't yet much discussion on that aspect.
> cc [~westonpace] [~lidavidm]



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