Skip to content

Fix nullable-dtype error when writing partitioned parquet data#8400

Merged
rjzamora merged 7 commits intodask:mainfrom
rjzamora:fix-nullable-partitioned
Dec 3, 2021
Merged

Fix nullable-dtype error when writing partitioned parquet data#8400
rjzamora merged 7 commits intodask:mainfrom
rjzamora:fix-nullable-partitioned

Conversation

@rjzamora
Copy link
Copy Markdown
Member

@rjzamora rjzamora commented Nov 18, 2021

Avoids duplicated pandas DataFrame objects within _write_partitioned (used for directory-partitioned to_parquet operations for the "pyarrow" engine). The current logic converts the original DataFrame partition into a pyarrow Table before writing, but then converts the Table back into a DataFrame for "hive" partitioning. This round trip approach is inefficient when the original DataFrame is still available for the groupby operation. The current approach also exposes a risk of loosing nullable-dtype information (which can be a serious problem for appended writes - see #8373).

Note that this PR will fix the dtype information stored in the metadata of partitioned-parquet files. However, appending will still break for older partitioned datasets that were produced with earlier versions of Dask. We will need to improve the "pyarrow" engine to coerce compatible-but-different dtypes to allow users to append to these older files.

filename,
partition_cols,
fs,
preserve_index,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

presumably we don't need to worry about the order changing since this is a private function?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right - I assumed explcit arguments were probably better since the funciton is private. However, I have no problem using kwargs if you think this is an unncessary risk

@jsignell
Copy link
Copy Markdown
Member

It looks like there are some genuine CI failures on this.

@rjzamora
Copy link
Copy Markdown
Member Author

rjzamora commented Nov 22, 2021

It looks like there are some genuine CI failures on this.

I am having a lot of trouble reproducing the failures locally, but I'll create the same ubuntu environment and see if I can reproduce :/

Huh... Still cannot reproduce those errors. CI is passing, but I would certainly like to understand why those tests were failing.

@scharlottej13
Copy link
Copy Markdown
Contributor

Hi @rjzamora, have you had a chance to look into this more?

@rjzamora
Copy link
Copy Markdown
Member Author

rjzamora commented Dec 1, 2021

rerun tests

@rjzamora
Copy link
Copy Markdown
Member Author

rjzamora commented Dec 2, 2021

have you had a chance to look into this more?

Thanks for the ping @scharlottej13! The CI errors we were originally seeing have not popped up again recently, and I have not been able to reproduce them locally. I wish I could say that I am sure the root cause was fixed elsewhere, but I'm just unsure :/

Overall, my suggestions is that we merge this. My gut tells me that the changes in this PR should be "correct", and that the test_create_metadata_file failure is likley explained by this existing issue. Also, the test_writing_parquet_with_kwargs failure is only for the "fatparquet" engine, which shouldn't be hitting the changes in this PR.

@rjzamora
Copy link
Copy Markdown
Member Author

rjzamora commented Dec 3, 2021

I plan to merge this tomorrow afternoon if there are no complaints (cc @jsignell - In case you wanted to take a final look)

@rjzamora rjzamora merged commit 21b99af into dask:main Dec 3, 2021
@rjzamora rjzamora deleted the fix-nullable-partitioned branch December 3, 2021 20:49
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Nullable types saved incorrectly in partitioned parquet files, leading to ValueError when appending

3 participants