New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Read Parquet directly into string[pyarrow] #9631
Comments
Thanks @mrocklin. This sounds sensible. Something like Was the dataset originally written with extension string dtypes? Ideally I'd like to see us respect roundtripping |
This one was not. This one is maintained by nyc tlc and I don't think that they (yet!) use Dask for data prep.
I would hope so, but I'm not sure that that's the case. In this case Dask/Pandas/Arrow aren't sure that the object dtypes are strings, they might be some other serialized object. In this case I'd like to explicitly tell the system "nope, this is definitely text data, please act accordingly". |
@jrbourbeau We have data written with pyarrow strings, python strings and categorical (3 different sets), that we can use to test this if needed. |
If I remember correctly (which probably don't), this may mean there is no way to do anything more efficient than using |
I've got to believe that when Arrow reads this dataset they are able to see "yup, this is text" and read it in appropriately. Spark also. @jorisvandenbossche any suggestions? |
I think that's right. See pandas-dev/pandas#42664 (comment) @mrocklin are you able to set I'm not sure if that has a trip from arrow -> python objects -> arrow, but hopefully it's zero-copy from arrow memory to pandas. |
Still no dice on the problem above: In [1]: import pandas as pd
In [2]: pd.option_context("string_storage", "pyarrow")
Out[2]: <pandas._config.config.option_context at 0x10295d690>
In [3]: import dask.dataframe as dd
...:
...: df = dd.read_parquet(
...: "s3://nyc-tlc/trip data/fhvhv_tripdata_2022-06.parquet",
...: split_row_groups=True,
...: use_nullable_dtypes=True,
...: )
...: df.dtypes
...:
Out[3]:
hvfhs_license_num object
dispatching_base_num object
originating_base_num object
request_datetime datetime64[ns]
on_scene_datetime datetime64[ns]
pickup_datetime datetime64[ns]
dropoff_datetime datetime64[ns]
PULocationID int64
DOLocationID int64
trip_miles float64
trip_time int64
base_passenger_fare float64
tolls float64
bcf float64
sales_tax float64
congestion_surcharge float64
airport_fee float64
tips float64
driver_pay float64
shared_request_flag object
shared_match_flag object
access_a_ride_flag object
wav_request_flag object
wav_match_flag object
dtype: object I think that the problem here is less about the |
I didn't test in combination with dask, but just using pandas, the following seems to work:
So it might be some issue in how dask is propagating those keywords to pandas? |
I was suspicious that this might be due to PyArrow writing metadata differently than however this data in the wild was written so I read the dataset with Pandas. I received the following: In [1]: import pandas as pd
In [2]: pd.option_context("string_storage", "pyarrow")
Out[2]: <pandas._config.config.option_context at 0x106be1720>
In [3]: %time df = pd.read_parquet("s3://nyc-tlc/trip data/fhvhv_tripdata_2022-0
...: 1.parquet", use_nullable_dtypes=True)
CPU times: user 10.3 s, sys: 4.89 s, total: 15.2 s
Wall time: 59.5 s
In [4]: df.dtypes
Out[4]:
hvfhs_license_num string
dispatching_base_num string
originating_base_num string
request_datetime datetime64[ns]
on_scene_datetime datetime64[ns]
pickup_datetime datetime64[ns]
dropoff_datetime datetime64[ns]
PULocationID Int64
DOLocationID Int64
trip_miles Float64
trip_time Int64
base_passenger_fare Float64
tolls Float64
bcf Float64
sales_tax Float64
congestion_surcharge Float64
airport_fee Float64
tips Float64
driver_pay Float64
shared_request_flag string
shared_match_flag string
access_a_ride_flag string
wav_request_flag string
wav_match_flag string
dtype: object
In [5]: df.hvfhs_license_num.head()
Out[5]:
0 HV0003
1 HV0003
2 HV0003
3 HV0003
4 HV0003
Name: hvfhs_license_num, dtype: string
In [6]: df.hvfhs_license_num.head().values
Out[6]:
<StringArray>
['HV0003', 'HV0003', 'HV0003', 'HV0003', 'HV0003']
Length: 5, dtype: string
In [9]: df.memory_usage(deep=True)
Out[9]:
Index 128
hvfhs_license_num 929350233
dispatching_base_num 929350233
originating_base_num 839144900
request_datetime 118012728
on_scene_datetime 118012728
pickup_datetime 118012728
dropoff_datetime 118012728
PULocationID 132764319
DOLocationID 132764319
trip_miles 132764319
trip_time 132764319
base_passenger_fare 132764319
tolls 132764319
bcf 132764319
sales_tax 132764319
congestion_surcharge 132764319
airport_fee 132764319
tips 132764319
driver_pay 132764319
shared_request_flag 855592278
shared_match_flag 855592278
access_a_ride_flag 855592278
wav_request_flag 855592278
wav_match_flag 855592278
dtype: int64 It looks like Pandas is properly getting the string dtype (🎉). It's not clear to me that it's using pyarrow here for storage. Those storage sizes seem large to me given the values. |
I think you want to use In [1]: import pandas as pd
In [2]: pd.set_option("mode.string_storage", "pyarrow")
In [3]: df = pd.read_parquet("s3://nyc-tlc/trip data/fhvhv_tripdata_2022-01.parquet", use_nullable_dtypes=True)
In [4]: df.hvfhs_license_num.dtype
Out[4]: string[pyarrow] |
Oh cool. And memory usage is lower in that case too. OK, I guess then this becomes more a question of how Dask could be leveraging this as well |
As an aside, but possibly related, in pandas 2.0 there will be another option
From a quick look, |
Following up here, #9617, which adds support for |
Okay, so with the current This works great for things like In [6]: import pandas as pd
In [7]: pd.set_option("mode.string_storage", "pyarrow") For Dask users on a single machine, this works great. We've seen some significant memory reductions by using pyarrow strings and it'd be good for Dask users to get those benefits by default (or as close to default as we think is reasonable). One option I see is inside Thinking a bit more ambitiously, I'm wondering if it makes sense for Dask check if |
From an exposure point of view, it would be very cool to see Dask opt into the For |
That's great to know. Given it's getting more attention, and there seem to be large performance benefits in lots of cases, I'm curious if switching to
My current rough understanding is, in general, |
Yeah definitely
And if pyarrow backed data needs to fallback on a slower path, the user should be warned
|
FYI for folks here, in #9719 I'm suggesting we extend With that PR, @mrocklin's original example looks like: In [1]: import dask.dataframe as dd
In [2]: df = dd.read_parquet(
...: "s3://nyc-tlc/trip data/fhvhv_tripdata_2022-06.parquet",
...: split_row_groups=True,
...: use_nullable_dtypes="pyarrow",
...: )
In [3]: df.dtypes
Out[3]:
hvfhs_license_num string
dispatching_base_num string
originating_base_num string
request_datetime timestamp[us][pyarrow]
on_scene_datetime timestamp[us][pyarrow]
pickup_datetime timestamp[us][pyarrow]
dropoff_datetime timestamp[us][pyarrow]
PULocationID int64[pyarrow]
DOLocationID int64[pyarrow]
trip_miles double[pyarrow]
trip_time int64[pyarrow]
base_passenger_fare double[pyarrow]
tolls double[pyarrow]
bcf double[pyarrow]
sales_tax double[pyarrow]
congestion_surcharge double[pyarrow]
airport_fee double[pyarrow]
tips double[pyarrow]
driver_pay double[pyarrow]
shared_request_flag string
shared_match_flag string
access_a_ride_flag string
wav_request_flag string
wav_match_flag string
dtype: object Note that In [4]: df.wav_match_flag.dtype
Out[4]: string[pyarrow] As a side not, @mroeschke I've been confused a few times where |
I'm excited about this. Two questions:
|
I care less about the name if there is a good path towards making this default |
Nice, glad to hear it
Hm I'm not sure I agree. I view the
I think both. I like the manual keyword here while we try things out, but long-term I think we'll want to add a |
The term nullable to me means "a data type that supports an NA or missing value". I agree that that's consistent with pyarrow strings and other pyarrow types, but it's not the first thing that comes to mind. If I'm a user and I'm saying "gosh, I want those fast arrow data types. How do I get those?" If I see "nullable" I'm not going to immediately say "Yes! that's clearly the fast arrow thing that I want" It feels a bit like putting bananas in the dairy section of a super market because they're white. Yes, they do fit. But no, it's not the term that I would normally search for them by. Maybe that doesn't make sense though. I don't know. Again, I care less if we're going to move this to be default. |
This is really interesting feedback coming from the pandas side and would help market "why should I use pyarrow" in pandas too. pandas (I) originally grouped the arrow types with the pandas (numpy based) nullable dtypes since they share the common and original motivation for the nullable dtypes - a dtype independent way to represent missing values. Hence why in pandas we also want to let to user choose arrow via nullable-related keywords e.g. The arrow types also happen to generally be more performant too so maybe they should be associated with more performance based keywords? e.g. |
@jrbourbeau I would consider this a bug. If it's easy to get a simple reproducer, it would be great to have as a pandas issue |
If pandas is using the term As a pandas user though, this does feel strange to me though. I understand how we got to "nullable" as a term here, but it doesn't feel right to me as someone who has just arrived at this situation. |
Opened pandas-dev/pandas#50099
I feel like there's a banana split joke to be had here, I just can find it... That said, it's a fair point.
@mroeschke how hardened is the relationship between "nullable" and the |
Also, if we're ok moving the default around in the future then I care much less. It seems like arrow support is pretty good these days. I'm hopeful that the developers and experts will use it, see that it works well, and then we'll switch the defaults around. The devs and experts won't care much about the name. |
As of 1.5 not really hardened. Their only association is just documenting that they both use |
Thanks @mroeschke. I saw online that meeting is open to the public, so I might drop by to say hi and represent Dask folks there |
Awesome! I put it on the meeting agenda so would be great to hear a representative pitch from the Dask side |
I would like to read parquet data directly into string pyarrow dtypes
So I'm trying this naively on a dataset:
This is especially important to me because in this case one row group is 10GB when stored naively. The data is somewhat unreadable in its current state on modest machines. I also suspect that I'm spending almost all of my time just creating and then destroying Python objects.
Any thoughts @rjzamora @ian-r-rose ?
The text was updated successfully, but these errors were encountered: