Skip to content

Add schema keyword argument to to_parquet#5150

Merged
mrocklin merged 15 commits intodask:masterfrom
birdsarah:fixes-inconsistent-schema-part-pyarrow
Jul 31, 2019
Merged

Add schema keyword argument to to_parquet#5150
mrocklin merged 15 commits intodask:masterfrom
birdsarah:fixes-inconsistent-schema-part-pyarrow

Conversation

@birdsarah
Copy link
Copy Markdown
Contributor

@birdsarah birdsarah commented Jul 24, 2019

Fixes #4194

This supercedes @mmccarty's PR #4851. It takes his tests but works with the new parquet code.

@mmccarty hope you don't mind me just doing this. was feeling motivated this morning.

  • Tests added / passed
  • Passes black dask / flake8 dask

@mrocklin
Copy link
Copy Markdown
Member

cc @rjzamora

@mrocklin
Copy link
Copy Markdown
Member

It looks like there is a failing test

@mmccarty
Copy link
Copy Markdown
Member

@birdsarah Thanks for taking this on! Feel free to have at it!

Have verified this test fails if code change is not present.
("partition_column", pa.int64()),
]
)
engine = "pyarrow"
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.

Let's just inline "pyarrow" rather than have an engine variable perhaps

birdsarah and others added 4 commits July 24, 2019 18:46
@birdsarah
Copy link
Copy Markdown
Contributor Author

@mmccarty sorry for all the pings. I've decided to take ownership properly and am building on your tests.

@birdsarah
Copy link
Copy Markdown
Contributor Author

@mrocklin the assortment of assertions that now are present (df.timestamps.equals and np.array_equal) are the only ones that I could get working. Any interest in different assertions would need to be done by someone else with more experience than me.

Note that setting tz to, say, 'US/Eastern' in place of 'UTC' and the timezone test fails. I have now spent more time than I have trying to debug this. I don't understand timezone support very well. The result is that the times come back out the wrong time. If anyone wants to jump in and fix, great. Otherwise we should note the limitations of timezone support somewhere - which should be written by someone who understands what's gone wrong.

@birdsarah
Copy link
Copy Markdown
Contributor Author

Need some help. I have a test failing on travis https://travis-ci.org/dask/dask/jobs/563346766#L1074 that does not fail locally. Any suggestions on how to fix gratefully received.

@rjzamora
Copy link
Copy Markdown
Member

@birdsarah Sorry I can't be very helpful now, but I should be able to take a closer look at this tomorrow morning. My quick suggestion would be to set gather_statistics=False in the read_parquet call (because you probably don't need statsitics/divisions for this test anyway).

@birdsarah
Copy link
Copy Markdown
Contributor Author

Thanks @rjzamora, I've given that a try. We'll see how it fares. I'd obviously prefer to know why there's a discrepancy between local and travis. The only obvious thing I can see is that travis has distributed pinned. But I can't imagine why that would affect this code path.

@birdsarah
Copy link
Copy Markdown
Contributor Author

Sigh. I'm back to an error I battled with extensively locally. Would appreciate a snippet of code from someone with the best way to test the timezone columns in and out.

@rjzamora
Copy link
Copy Markdown
Member

I don't have a great understanding of timestamp types, but the difference in indices before and after the partition seems to be causing the issue. Resetting the index seems to work for me:

assert df.timestamps.equals(
    ddf_after_write.timestamps.compute().reset_index(drop=True)
)

Note timezones are not supported. They come out from read tz-naive.
@birdsarah
Copy link
Copy Markdown
Contributor Author

Fixed this up so I hope it reliably passes. It's a little verbose but NaT != NaT so this seems clear at least.

Also now documents that timezones are a problem. They come out of the write-read cycle setup in the test as timezone naive objects.

@mmccarty
Copy link
Copy Markdown
Member

@birdsarah Wow! You've been busy on this one! Sorry I was slow to respond about the tests. Looks like you've got it now. I'll get caught up on this PR and close the old one.

This was referenced Jul 26, 2019
@mmccarty
Copy link
Copy Markdown
Member

@birdsarah It would be helpful to get this fix into the next release. If you can't get back to it soon and want some help let me know. I can try to find help getting that unit test wrapped up.

@mrocklin
Copy link
Copy Markdown
Member

@rjzamora is taking a look

Also @birdsarah I notice a number of black formatting commits. You may be interested in https://docs.dask.org/en/latest/develop.html#code-formatting

Which recommends the following:

pip install pre-commit
pre-commit install

@birdsarah
Copy link
Copy Markdown
Contributor Author

working on it now

@rjzamora
Copy link
Copy Markdown
Member

rjzamora commented Jul 29, 2019

PR#5157 now includes these changes (or similar), and hopefully the CI will pass... If @birdsarah does have time to push this through, I can certainly sync again with this PR after it is merged

[EDIT: Scratch that - 5157 does not have correct timestamp assertions at the moment.]

@birdsarah
Copy link
Copy Markdown
Contributor Author

This should be it. I have no idea why locally these tests would pass but not on CI. Things should be super explicit now.

@birdsarah
Copy link
Copy Markdown
Contributor Author

This is now passing. Let me know if you'd like me to add the compute test.

@rjzamora
Copy link
Copy Markdown
Member

This is now passing. Let me know if you'd like me to add the compute test.

No need to add compute here. I will add it to #5157 if it really is unavoidable.

The changes look good to me. My only suggestion might be to use shorter names for the new tests - but that's probably just a personal preference

@birdsarah
Copy link
Copy Markdown
Contributor Author

I'm afraid I prefer long test names. So we'll have to agree to disagree or someone can point me to a style guide.

@mmccarty
Copy link
Copy Markdown
Member

mmccarty commented Jul 29, 2019 via email

@mrocklin
Copy link
Copy Markdown
Member

What's the status here. Is this good to go in?

@rjzamora
Copy link
Copy Markdown
Member

+1 from me

@mrocklin
Copy link
Copy Markdown
Member

Great! Merging in. Thanks @birdsarah and @mmccarty for the work and @rjzamora for review.

@ldacey
Copy link
Copy Markdown

ldacey commented Mar 15, 2020

Is there a way to print out the pyarrow field list / schema so I can save it in a file?

Specifically, I'd like to:

  1. Clean up my dataframe and call schema = pa.Schema.from_pandas(df)
  2. Print out a list of pa.fields like this:
fields = [
    pa.field('id', pa.int64()),
    pa.field('secondaryid', pa.int64()),
    pa.field('date', pa.timestamp('ms')), 
    pa.field('status', pa.dictionary(pa.int8(), pa.string(), ordered=False),
]
  1. Edit those fields, if needed, and then save it in a file
  2. Refer to the list of fields when a save a parquet file (to help eliminate issues where a partition might have all null values or mixed dtypes for a certain column) when reading a bunch of files in with Dask

Some files have hundreds of columns so I don't want to categorize each one manually. The inference from Schema.from_pandas() is good for 99% of the columns, there are just a few I need to tweak and it'd be nice to have a hard coded reference (similar to a create table script in SQL)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Inconsistent Schema with partitioned pyarrow files

5 participants