Avoid apply in _compute_sum_of_squares for groupby std agg#6186
Avoid apply in _compute_sum_of_squares for groupby std agg#6186jcrist merged 3 commits intodask:masterfrom
Conversation
|
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. |
TomAugspurger
left a comment
There was a problem hiding this comment.
Just one question. Looks good otherwise.
|
@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?] |
|
Oh - It looks like Ashwin pointed out the |
| 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`. |
There was a problem hiding this comment.
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.
|
LGTM, thanks @rjzamora. Merging. |
This PR addresses #6034 by avoiding the
applycall in_compute_sum_of_squaresforstdgroupby aggregations.Motivation: The following does not currently work with
dask_cudf, because thedask.dataframeimplementation uses groupby-apply (which does not exhibit "correct" behavior for all cases in cudf):Note that this change also improves pandas-backed
dask.dataframeperformance: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)black dask/flake8 dask