[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=17408162#comment-17408162 ] David Li commented on ARROW-13813: -- {quote}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. {quote} I agree, this would be useful to support. I assume once we have a node for writing, this will be easier. {quote}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? {quote} On Windows there are more invalid path characters to consider, but I agree, we should raise an error instead of writing unreadable data. (I think for auto-encoding values, that can be done by bindings and doesn't necessarily have to be done in C++? e.g. the Python/R bindings could offer an option to wrap string partition fields in a URL-encode pass.) > [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)
[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)
[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=17407593#comment-17407593 ] Weston Pace commented on ARROW-13813: - I'll create the new JIRA > [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)
[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=17407592#comment-17407592 ] Weston Pace commented on ARROW-13813: - That can be a new JIRA. I think we can keep this open as it is to discuss and track using those kernels to provide utility in a partitioning object. > [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)
[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=17407571#comment-17407571 ] David Li commented on ARROW-13813: -- Then should we make this JIRA about implementing a URL-encode (and probably a corresponding URL-decode) kernel? > [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)
[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=17407565#comment-17407565 ] Weston Pace commented on ARROW-13813: - I think my only concern is that this is something the user should be able to easily do themselves using the compute stuff. They could use a scanner to read in their data, project the offending column to an encoding kernel, and then partition on the projected column. However, since we already have segment encoding in partition objects it seems straightforward enough to provide. It might be a good project to pair with ARROW-11378 if someone is looking for some good beginner C++ tasks. > [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)
[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=17407292#comment-17407292 ] David Li commented on ARROW-13813: -- I would think we should be able to. > [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)