Skip to content

Commit a7b7961

Browse files
authored
Add draft of cudf groupby aggregations (#14)
1 parent 2bfe8b3 commit a7b7961

File tree

1 file changed

+155
-0
lines changed

1 file changed

+155
-0
lines changed
Lines changed: 155 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,155 @@
1+
---
2+
layout: post
3+
title: Building GPU Groupby-Aggregations for Dask
4+
author: Matthew Rocklin
5+
tags: [dataframe, GPU, RAPIDS]
6+
theme: twitter
7+
---
8+
9+
{% include JB/setup %}
10+
11+
Summary
12+
-------
13+
14+
We've sufficiently aligned Dask DataFrame and cuDF to get groupby aggregations
15+
like the following to work well.
16+
17+
```python
18+
df.groupby('x').y.mean()
19+
```
20+
21+
This post describes the kind of work we had to do as a model for future
22+
development.
23+
24+
25+
Plan
26+
----
27+
28+
As outlined in a previous post, [Dask, Pandas, and GPUs: first
29+
steps](../../../2019/01/13/dask-cudf-first-steps.html), our plan to produce
30+
distributed GPU dataframes was to combine [Dask
31+
DataFrame](https://docs.dask.org/en/latest/dataframe.html) with
32+
[cudf](https://rapids.ai). In particular, we had to
33+
34+
- change Dask DataFrame so that it would parallelize not just around the
35+
Pandas DataFrames that it works with today, but around anything that looked
36+
enough like a Pandas DataFrame
37+
- change cuDF so that it would look enough like a Pandas DataFrame to fit
38+
within the algorithms in Dask DataFrame
39+
40+
Changes
41+
-------
42+
43+
On the Dask side this mostly meant replacing
44+
45+
- Replacing `isinstance(df, pd.DataFrame)` checks with `is_dataframe_like(df)`
46+
checks (after defining a suitable
47+
`is_dataframe_like`/`is_series_like`/`is_index_like` functions
48+
- Avoiding some more exotic functionality in Pandas, and instead trying to
49+
use more common functionality that we can expect to be in most DataFrame
50+
implementations
51+
52+
On the cuDF side this means making dozens of tiny changes to align the cuDF API
53+
to the Pandas API, and to add in missing features.
54+
55+
- **Dask Changes:**
56+
- [Remove explicit pandas checks and provide cudf lazy registration #4359](https://github.com/dask/dask/pull/4359)
57+
- [Replace isinstance(..., pandas) with is_dataframe_like #4375](https://github.com/dask/dask/pull/4375)
58+
- [Add has_parallel_type](https://github.com/dask/dask/pull/4395)
59+
- [Lazily register more cudf functions and move to backends file #4396](https://github.com/dask/dask/pull/4396)
60+
- [Avoid checking against types in is_dataframe_like #4418](https://github.com/dask/dask/pull/4418)
61+
- [Replace cudf-specific code with dask-cudf import #4470](https://github.com/dask/dask/pull/4470)
62+
- [Avoid groupby.agg(callable) in groupby-var #4482](https://github.com/dask/dask/pull/4482) -- this one is notable in that by simplifying our Pandas usage we actually got a significant speedup on the Pandas side.
63+
- **cuDF Changes:**
64+
- [Build DataFrames from CUDA array libraries #529](https://github.com/rapidsai/cudf/issues/529)
65+
- [Groupby AttributeError](https://github.com/rapidsai/cudf/issues/534)
66+
- [Support comparison operations on Indexes #556](https://github.com/rapidsai/cudf/issues/556)
67+
- [Support byte ranges in read_csv (and other formats) #568](https://github.com/rapidsai/cudf/issues/568):w
68+
- [Allow "df.index = some_index" #824](https://github.com/rapidsai/cudf/issues/824)
69+
- [Support indexing on groupby objects #828](https://github.com/rapidsai/cudf/issues/828)
70+
- [Support df.reset_index(drop=True) #831](https://github.com/rapidsai/cudf/issues/831)
71+
- [Add Series.groupby #879](https://github.com/rapidsai/cudf/issues/879)
72+
- [Support Dataframe/Series groupby level=0 #880](https://github.com/rapidsai/cudf/issues/880)
73+
- [Implement division on DataFrame objects #900](https://github.com/rapidsai/cudf/issues/900)
74+
- [Groupby objects aren't indexable by column names #934](https://github.com/rapidsai/cudf/issues/934)
75+
- [Support comparisons on index operations #937](https://github.com/rapidsai/cudf/issues/937)
76+
- [Add DataFrame.rename #944](https://github.com/rapidsai/cudf/issues/944)
77+
- [Set the index of a dataframe/series #967](https://github.com/rapidsai/cudf/issues/967)
78+
- [Support concat(..., axis=1) #968](https://github.com/rapidsai/cudf/issues/968)
79+
- [Support indexing with a pandas index from columns #969](https://github.com/rapidsai/cudf/issues/969)
80+
- [Support indexing a dataframe with another boolean dataframe #970](https://github.com/rapidsai/cudf/issues/970)
81+
82+
I don't really expect anyone to go through all of those issues, but my hope is
83+
that by skimming over the issue titles people will get a sense for the kinds of
84+
changes we're making here. It's a large number of small things.
85+
86+
Also, kudos to [Thomson Comer](https://github.com/thomcom) who solved most of
87+
the cuDF issues above.
88+
89+
90+
There are still some pending issues
91+
-----------------------------------
92+
93+
- [Square Root #1055](https://github.com/rapidsai/cudf/issues/1055), needed for groupby-std
94+
- [cuDF needs multi-index support for columns #483](https://github.com/rapidsai/cudf/issues/483), needed for:
95+
96+
```python
97+
gropuby.agg({'x': ['sum', mean'], 'y': ['min', 'max']})
98+
```
99+
100+
101+
But things mostly work
102+
----------------------
103+
104+
But generally things work pretty well today:
105+
106+
```python
107+
In [1]: import dask_cudf
108+
109+
In [2]: df = dask_cudf.read_csv('yellow_tripdata_2016-*.csv')
110+
111+
In [3]: df.groupby('passenger_count').trip_distance.mean().compute()
112+
Out[3]: <cudf.Series nrows=10 >
113+
114+
In [4]: _.to_pandas()
115+
Out[4]:
116+
0 0.625424
117+
1 4.976895
118+
2 4.470014
119+
3 5.955262
120+
4 4.328076
121+
5 3.079661
122+
6 2.998077
123+
7 3.147452
124+
8 5.165570
125+
9 5.916169
126+
dtype: float64
127+
```
128+
129+
Experience
130+
----------
131+
132+
First, most of this work was handled by the cuDF developers (which may be
133+
evident from the relative lengths of the issue lists above). When we started
134+
this process it felt like a never-ending stream of tiny issues. We weren't
135+
able to see the next set of issues until we had finished the current set.
136+
Fortunately, most of them were pretty easy to fix. Additionally, as we went
137+
on, it seemed to get a bit easier over time.
138+
139+
Additionally, lots of things work other than groupby-aggregations as a result
140+
of the changes above. From the perspective of someone accustomed to Pandas,
141+
The cuDF library is starting to feel more reliable. We hit missing
142+
functionality less frequently when using cuDF on other operations.
143+
144+
145+
What's next?
146+
------------
147+
148+
More recently we've been working on the various join/merge operations in Dask
149+
DataFrame like indexed joins on a sorted column, joins between large and small
150+
dataframes (a common special case) and so on. Getting these algorithms from
151+
the mainline Dask DataFrame codebase to work with cuDF is resulting in a
152+
similar set of issues to what we saw above with groupby-aggregations, but so
153+
far the list is much smaller. We hope that this is a trend as we continue on
154+
to other sets of functionality into the future like I/O, time-series
155+
operations, rolling windows, and so on.

0 commit comments

Comments
 (0)