Support non-uniform categoricals#1877
Conversation
|
This looks cool. It would be nice to remove this roadblock when using categoricals with dask.dataframe. |
|
What is the status here? @jcrist are you waiting on feedback? Please let me know if this is blocking on me. |
|
No, I'm still working on this. |
This adds support for non-consistent categoricals in each partition.
Both `get_dummies` and `pivot_table` error properly if categories are not fully known.
Throw a NotImplementedError if the categories are unknown. An alternative may be to compute immediately, or to return a lazy Index object.
Test common operations work properly
We do this because pandas frequently has inconsistent dtypes in operations with empty dataframes. We may want to move this into `methods.concat`, but at the same time that needs to be able to use empty frames to ensure partitions match the desired schema. This all is pretty mucky stuff, hopefully can find a cleaner way.
Reading directly into a categorical dtype from `read_csv` now returns an unknown categorical.
- `known` is whether the categories are known - `as_known` converts unknown -> known - `as_unknown` converts known -> unknown
- pin conda to fix conda bug? - skip csv test in pandas < 0.19.2
Previously categorical values were hashed using just their codes. This meant that the hash value depended on the ordering of the categories, rather than on the values the series represented. This caused problems in dask, where different partitions might have different categorical mappings. This PR makes the hashing dependent on the values the categorical represents, rather than on the codes. The categories are first hashed, and then the codes are remapped to the hashed values. This is slightly slower than before (still need to hash the categories, where we didn't before), but allows for more consistent hashing. Related to this work in dask: dask/dask#1877. Author: Jim Crist <crist042@umn.edu> Closes #15143 from jcrist/categories_hash_consistently and squashes the following commits: f1aea13 [Jim Crist] Address comments 7878c55 [Jim Crist] Categoricals hash consistently
Due to changes to `concat`, python ints no longer convert to int32 on int32 systems when concatenated
|
Ok, this should be ready for review. A quick walk through of what's added here: This adds support for different partitions having different categorical mappings. This is accomplished through Changes to metadata:Categorical columns/indices now have two states:
Changes to the categorical accessorTo query and convert between these states, following has been added to the categorical accessor:
If the categories are known, then Changes to concatConcatenation of dask dataframes with different categoricals is now supported. For each column/index in a dataframe, the output dask column/index will have known categories only if all categories are known. Otherwise the categories will be unknown. After a call to concat, each partition is guaranteed to match the metadata on the containing dask dataframe. This fixes a bug from before where in certain cases partitions could have different columns/datatypes. Changes to
|
|
And a quick demo using the airline dataset: In [1]: import pandas as pd
In [2]: dtypes = pd.read_csv('1987.csv', dtype={'Origin': 'category', 'Dest': 'category'}).dtypes.to_dict()
In [3]: import dask.dataframe as dd
# The dtypes to `Origin` and `Dest` are `category`, so they're
# read directly in as categoricals
In [4]: ddf = dd.read_csv('198*.csv', dtype=dtypes)
In [5]: ddf
Out[5]: dd.DataFrame<from-de..., npartitions=18>
In [6]: ddf.Origin.dtype
Out[6]: category
In [7]: ddf.Origin.cat.known # categories are unknown at this point
Out[7]: False
In [8]: worst_avg_delay = ddf.DepDelay.groupby(ddf.Origin).mean().nlargest(10)
In [9]: worst_avg_delay.index.dtype
Out[9]: category
In [10]: worst_avg_delay.index.cat.known
Out[10]: False
In [11]: worst_avg_delay.compute()
Out[11]:
Origin
PIR 85.000000
ABI 36.500000
EGE 32.363636
BFI 28.000000
ROP 24.700000
ACV 19.740223
SUN 19.000000
YAP 16.902482
RDD 15.724234
HDN 14.147350
Name: DepDelay, dtype: float64
# Convert unknown categoricals to known
In [12]: known_origin = ddf.Origin.cat.as_known()
In [13]: known_origin.cat.known
Out[13]: True
In [14]: known_origin.cat.categories
Out[14]:
Index(['ABE', 'ABQ', 'ACV', 'AGS', 'ALB', 'ALO', 'AMA', 'ANC', 'APF', 'ATL',
...
'PIR', 'ACY', 'GST', 'DET', 'BFI', 'ROP', 'ABI', 'TVC', 'EGE', 'SUN'],
dtype='object', length=246) |
|
One question (would be nice to confirm in the docs): In [300]: s = dd.from_pandas(pd.Series(pd.Categorical(['c', 'b', 'b', 'a']), name='name'), npartitions=2).cat.as_unknown()
In [301]: s.cat.codes.compute()
Out[301]:
0 2
1 1
2 1
3 0
dtype: int8 |
|
Ooh, that's a good point, I didn't think of that one. This is actually True only for categoricals that are known. For unknown categoricals, we'd need need a unifying step, which can't be done without a full dataset traversal. My preference is to error in this case, rather than doing something inefficient. Thanks for coming up with all the things I hadn't thought of yet. This is a good review :). |
Cleanups to categorical accessor: - Refactor internal design of accessor - Accessor key names include accessor name - Move categorical accessor to `categorical.py` - Categorical accessor works for indices Cleanups to categorize: - Categorize of categorical is idempotent - Categorize transforms unknown -> known categoricals if requested - Categorize works on index
- Categorical methods present on categorical index - Test methods work on categorical index - Methods present in `dir` of index if present - Access to `cat` and `str` fail if not proper dtype
Previously non-Series/Index output from accessor methods would wrap their return values in calls to `pd.Series`. This resulted in inconsistencies between pandas and dask. Since we now support arrays with unknown chunks, we can return the proper type. Also forward the `dt` properties for datetime indices through the accessor, instead of adding methods on directly. Also improved error messages for categorical failures.
|
I believe all comments have been addressed. One new change that people may want to look at: previously we'd wrap all output from the accessors in calls to |
|
Failure is due to groupby not supporting grouping by array. Looks like an easy fix (we already support conversion). It's late here, will fix tomorrow. |
|
FYI, I won't have a chance to review this any more closely till next week, so I'm +1 to merge when you're ready. Great work!
On Fri, Jan 20, 2017 at 6:26 PM -0600, "Jim Crist" <notifications@github.com> wrote:
Failure is due to groupby not supporting grouping by array. Looks like an easy fix (we already support conversion). It's late here, will fix tomorrow.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub, or mute the thread.
|
|
So it turns out that having the accessor's return arrays (when they do in pandas) isn't the best thing, because we lose the division information (we actually have a groupby test with an accessor and known divisions). So I need to revert the behavior here :(. Q: Previously, when an accessor returns a numpy array, we'd wrap it in the same type as the partition type (so for Right now I'm leaning towards always wrapping numpy array return values in Series. Thoughts @sinhrks, @TomAugspurger? |
Revert to previous behavior, where `x.accessor.foo` returns the same type as `x` if the pandas version returns a numpy array. If `x` is a series, the output series will have the same index as `x`. This is to allow for broadcasting to work properly.
|
Ok, I reverted to the previous behavior, where After further though, I think this is for the best. For many operations, pandas treats arrays and indices effectively as equivalent (e.g. not aligning on index beforehand), making them a better match than series. For the one series operation that returns a numpy array ( |
|
All tests passed, I think this is good to go now. |
you mean |
Oop, yes, that's what I meant.
Many of the index methods/properties that match those on the |
|
@jcrist sure makes sense. I think pandas will soon also simply return |
|
I just gave this PR a quick test drive and was quite pleased with the experience. |
|
I'm going to merge this now, and add docs in a follow-up PR. I think it'd be good to get this on master now so others can try it out easier. Any changes/additional features can be added as needed later. |
|
I'm glad to see this in. Thanks taking on a large task like this @jcrist . |
Previously categorical values were hashed using just their codes. This meant that the hash value depended on the ordering of the categories, rather than on the values the series represented. This caused problems in dask, where different partitions might have different categorical mappings. This PR makes the hashing dependent on the values the categorical represents, rather than on the codes. The categories are first hashed, and then the codes are remapped to the hashed values. This is slightly slower than before (still need to hash the categories, where we didn't before), but allows for more consistent hashing. Related to this work in dask: dask/dask#1877. Author: Jim Crist <crist042@umn.edu> Closes pandas-dev#15143 from jcrist/categories_hash_consistently and squashes the following commits: f1aea13 [Jim Crist] Address comments 7878c55 [Jim Crist] Categoricals hash consistently
This adds support for non-uniform categoricals (different categories in each partition) to
dask.dataframe. Currently this is written to only support pandas 0.19.0. It could be made backwards compatible, but we may also want to discuss dropping support for < 0.19.0.Fixes #1836.
TODO:
get_dummiespivot_tableconcatappendmergeread_csv(as per ENH: parse categoricals in read_csv pandas-dev/pandas#13406)hash_pandas_objectis categorical-mapping agnosticMaybe make backwards compatible with pandas 0.18?