Skip to content

Avoid apply in _compute_sum_of_squares for groupby std agg#6186

Merged
jcrist merged 3 commits intodask:masterfrom
rjzamora:avoid-aply
May 11, 2020
Merged

Avoid apply in _compute_sum_of_squares for groupby std agg#6186
jcrist merged 3 commits intodask:masterfrom
rjzamora:avoid-aply

Conversation

@rjzamora
Copy link
Copy Markdown
Member

@rjzamora rjzamora commented May 8, 2020

This PR addresses #6034 by avoiding the apply call in _compute_sum_of_squares for std groupby aggregations.

Motivation: The following does not currently work with dask_cudf, because the dask.dataframe implementation uses groupby-apply (which does not exhibit "correct" behavior for all cases in cudf):

import cudf, dask_cudf

df = cudf.DataFrame({'a': [1,1,2,2],'b': [4,5,6,10]})
ddf = dask_cudf.from_cudf(df, npartitions=2)
ddf.groupby('a').agg({'b':['mean','std']}).compute()

Note that this change also improves pandas-backed dask.dataframe performance:

import cudf, dask_cudf
import dask.dataframe as dd
import pandas as pd
import numpy as np

size = 100_000_000
df = pd.DataFrame({'a': np.random.randint(10, size=size),'b': np.random.randint(10, size=size)})
ddf = dd.from_pandas(df, npartitions=2)

%timeit ddf.groupby('a').agg({'b':['mean','std']}).compute()

Master: 3.46 s ± 71.5 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)
This PR: 1.74 s ± 36.9 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

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

@mrocklin
Copy link
Copy Markdown
Member

mrocklin commented May 8, 2020

Oh great! I'm glad to see that it was possible to find a solution to this that used only more efficient Pandas/cudf API. Seeing the speedup on pandas is great, and knowing that it allows cudf operations to work at all is awesome. This seems to pass tests, so I'm +1, but I figure we wait a day in case someone like @TomAugspurger has thoughts.

Copy link
Copy Markdown
Member

@TomAugspurger TomAugspurger left a comment

Choose a reason for hiding this comment

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

Just one question. Looks good otherwise.

@rjzamora
Copy link
Copy Markdown
Member Author

rjzamora commented May 8, 2020

@shwina - Let me know if you have any thoughts/advice on how I am using cudf/groupby here. [EDIT: More specifically, is there a "public" attribute I should use instead?]

@rjzamora
Copy link
Copy Markdown
Member Author

rjzamora commented May 8, 2020

Oh - It looks like Ashwin pointed out the keys attribute here.

def _compute_sum_of_squares(grouped, column):
base = grouped[column] if column is not None else grouped
return base.apply(lambda x: (x ** 2).sum())
# Note: CuDF cannot use `groupby.apply`.
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.

Just a clarification, we can use groupby.apply but we should avoid it for both Pandas and cuDF whenever possible as it goes through a slower iteration based code path.

@jcrist
Copy link
Copy Markdown
Member

jcrist commented May 11, 2020

LGTM, thanks @rjzamora. Merging.

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.

5 participants