[
https://issues.apache.org/jira/browse/ARROW-7706?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17403459#comment-17403459
]
Weston Pace edited comment on ARROW-7706 at 8/24/21, 2:26 AM:
--------------------------------------------------------------
I've added some customization here in
https://github.com/apache/arrow/pull/10955 via "existing_data_behavior".
This will provide the options...
* kError - Raise an error if there are any files or directories in `base_dir`
(the new default)
* kOverwriteOrIgnore - Existing files will be ignored unless the filename is
one of those chosen by the dataset writer in which case they will be
overwritten (the old default)
* kDeleteMatchingPartitions - This is similar to the dynamic partition
overwrite mode in spark. The first time a directory is written to it will
delete any existing data.
Compared to the previous disucssion:
* Append - I think opening up append is a can of worms I'd rather avoid
* Error if partition exists - There is no good way to only error if a file
would be written to. By the time we figured that out we'd be halfway into the
write operation and you'd end up with a partially written dataset. I'm hoping
"error if there is any data there at all" will be sufficient.
Feedback on this approach is welcomed!
was (Author: westonpace):
I've added some customization here in
https://github.com/apache/arrow/pull/10955 via "existing_data_behavior".
This will provide the options...
* kError - Raise an error if there are any files or directories in `base_dir`
(the new default)
* kOverwriteOrIgnore - Existing files will be ignored unless the filename is
one of those chosen by the dataset writer in which case they will be
overwritten (the old default)
* kDeleteMatchingPartitions - This is similar to the dynamic partition
overwrite mode in parquet. The first time a directory is written to it will
delete any existing data.
Compared to the previous disucssion:
* Append - I think opening up append is a can of worms I'd rather avoid
* Error if partition exists - There is no good way to only error if a file
would be written to. By the time we figured that out we'd be halfway into the
write operation and you'd end up with a partially written dataset. I'm hoping
"error if there is any data there at all" will be sufficient.
Feedback on this approach is welcomed!
> [Python] saving a dataframe to the same partitioned location silently doubles
> the data
> --------------------------------------------------------------------------------------
>
> Key: ARROW-7706
> URL: https://issues.apache.org/jira/browse/ARROW-7706
> Project: Apache Arrow
> Issue Type: Bug
> Components: Python
> Affects Versions: 0.15.1
> Reporter: Tsvika Shapira
> Priority: Major
> Labels: dataset, dataset-parquet-write, parquet
>
> When a user saves a dataframe:
> {code:python}
> df1.to_parquet('/tmp/table', partition_cols=['col_a'], engine='pyarrow')
> {code}
> it will create sub-directories named "{{a=val1}}", "{{a=val2}}" in
> {{/tmp/table}}. Each of them will contain one (or more?) parquet files with
> random filenames.
> If a user runs the same command again, the code will use the existing
> sub-directories, but with different (random) filenames. As a result, any data
> loaded from this folder will be wrong - each row will be present twice.
> For example, when using
> {code:python}
> df1.to_parquet('/tmp/table', partition_cols=['col_a'], engine='pyarrow') #
> second time
> df2 = pd.read_parquet('/tmp/table', engine='pyarrow')
> assert len(df1) == len(df2) # raise an error{code}
> This is a subtle change in the data that can pass unnoticed.
>
> I would expect that the code will prevent the user from using an non-empty
> destination as partitioned target. an overwrite flag can also be useful.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)