diff --git a/docs/cudf/source/user_guide/api_docs/groupby.rst b/docs/cudf/source/user_guide/api_docs/groupby.rst index 80811efa33f..ca29087cbf9 100644 --- a/docs/cudf/source/user_guide/api_docs/groupby.rst +++ b/docs/cudf/source/user_guide/api_docs/groupby.rst @@ -68,7 +68,6 @@ Computations / descriptive stats GroupBy.std GroupBy.sum GroupBy.var - GroupBy.corr GroupBy.cov The following methods are available in both ``SeriesGroupBy`` and @@ -81,6 +80,7 @@ application to columns of a specific data type. :toctree: api/ DataFrameGroupBy.bfill + DataFrameGroupBy.corr DataFrameGroupBy.count DataFrameGroupBy.cumcount DataFrameGroupBy.cummax @@ -102,5 +102,6 @@ The following methods are available only for ``SeriesGroupBy`` objects. .. autosummary:: :toctree: api/ + SeriesGroupBy.corr SeriesGroupBy.nunique SeriesGroupBy.unique diff --git a/python/cudf/cudf/core/groupby/groupby.py b/python/cudf/cudf/core/groupby/groupby.py index 1646c5042fd..3cfbd1d736a 100644 --- a/python/cudf/cudf/core/groupby/groupby.py +++ b/python/cudf/cudf/core/groupby/groupby.py @@ -8,7 +8,7 @@ import warnings from collections import abc from functools import cached_property -from typing import TYPE_CHECKING, Any, Iterable +from typing import TYPE_CHECKING, Any, Iterable, Literal import cupy as cp import numpy as np @@ -306,6 +306,18 @@ def __iter__(self): grouped_values[offsets[i] : offsets[i + 1]], ) + def __len__(self) -> int: + return self.ngroups + + @property + def ngroups(self) -> int: + _, offsets, _, _ = self._grouped() + return len(offsets) - 1 + + @property + def ndim(self) -> int: + return self.obj.ndim + @property def dtypes(self): """ @@ -457,10 +469,20 @@ def size(self): ) @_performance_tracking - def cumcount(self): + def cumcount(self, ascending: bool = True): """ Return the cumulative count of keys in each group. + + Parameters + ---------- + ascending : bool, default True + If False, number in reverse, from length of group - 1 to 0. + Currently not supported """ + if ascending is not True: + raise NotImplementedError( + "ascending is currently not implemented." + ) return ( cudf.Series( cudf.core.column.column_empty( @@ -527,7 +549,7 @@ def _groupby(self): ) @_performance_tracking - def agg(self, func): + def agg(self, func, *args, engine=None, engine_kwargs=None, **kwargs): """ Apply aggregation(s) to the groups. @@ -615,6 +637,22 @@ def agg(self, func): 1 1.5 1.75 2.0 2.0 2 3.0 3.00 1.0 1.0 """ + if engine is not None: + raise NotImplementedError( + "engine is non-functional and added for compatibility with pandas" + ) + if engine_kwargs is not None: + raise NotImplementedError( + "engine_kwargs is non-functional added for compatibility with pandas" + ) + if args: + raise NotImplementedError( + "Passing args to func is currently not supported." + ) + if kwargs: + raise NotImplementedError( + "Passing kwargs to func is currently not supported." + ) column_names, columns, normalized_aggs = self._normalize_aggs(func) orig_dtypes = tuple(c.dtype for c in columns) @@ -935,12 +973,13 @@ def tail(self, n: int = 5, *, preserve_order: bool = True): ) @_performance_tracking - def nth(self, n): + def nth(self, n, dropna: Literal["any", "all", None] = None): """ Return the nth row from each group. """ - - self.obj["__groupbynth_order__"] = range(0, len(self.obj)) + if dropna is not None: + raise NotImplementedError("dropna is not currently supported.") + self.obj["__groupbynth_order__"] = range(0, len(self.obj)) # type: ignore[index] # We perform another groupby here to have the grouping columns # be a part of dataframe columns. result = self.obj.groupby(self.grouping.keys).agg(lambda x: x.nth(n)) @@ -1423,13 +1462,13 @@ def _post_process_chunk_results( @_performance_tracking def apply( - self, function, *args, engine="auto", include_groups: bool = True + self, func, *args, engine="auto", include_groups: bool = True, **kwargs ): """Apply a python transformation function over the grouped chunk. Parameters ---------- - function : callable + func : callable The python transformation function that will be applied on the grouped chunk. args : tuple @@ -1452,6 +1491,9 @@ def apply( When True, will attempt to apply ``func`` to the groupings in the case that they are columns of the DataFrame. In the future, this will default to ``False``. + kwargs : dict + Optional keyword arguments to pass to the function. + Currently not supported Examples -------- @@ -1528,13 +1570,17 @@ def mult(df): dtype: int64 """ + if kwargs: + raise NotImplementedError( + "Passing kwargs to func is currently not supported." + ) if self.obj.empty: - if function in {"count", "size", "idxmin", "idxmax"}: + if func in {"count", "size", "idxmin", "idxmax"}: res = cudf.Series([], dtype="int64") else: res = self.obj.copy(deep=True) res.index = self.grouping.keys - if function in {"sum", "product"}: + if func in {"sum", "product"}: # For `sum` & `product`, boolean types # will need to result in `int64` type. for name, col in res._data.items(): @@ -1542,20 +1588,20 @@ def mult(df): res._data[name] = col.astype("int") return res - if not callable(function): - raise TypeError(f"type {type(function)} is not callable") + if not callable(func): + raise TypeError(f"type {type(func)} is not callable") group_names, offsets, group_keys, grouped_values = self._grouped( include_groups=include_groups ) if engine == "auto": - if _can_be_jitted(grouped_values, function, args): + if _can_be_jitted(grouped_values, func, args): engine = "jit" else: engine = "cudf" if engine == "jit": result = self._jit_groupby_apply( - function, + func, group_names, offsets, group_keys, @@ -1564,7 +1610,7 @@ def mult(df): ) elif engine == "cudf": result = self._iterative_groupby_apply( - function, + func, group_names, offsets, group_keys, @@ -1744,12 +1790,14 @@ def _broadcast(self, values: cudf.Series) -> cudf.Series: return values @_performance_tracking - def transform(self, function): + def transform( + self, func, *args, engine=None, engine_kwargs=None, **kwargs + ): """Apply an aggregation, then broadcast the result to the group size. Parameters ---------- - function: str or callable + func: str or callable Aggregation to apply to each group. Note that the set of operations currently supported by `transform` is identical to that supported by the `agg` method. @@ -1778,18 +1826,35 @@ def transform(self, function): -------- agg """ - if not (isinstance(function, str) or callable(function)): + if engine is not None: + raise NotImplementedError( + "engine is non-functional and added for compatibility with pandas" + ) + if engine_kwargs is not None: + raise NotImplementedError( + "engine_kwargs is non-functional added for compatibility with pandas" + ) + if args: + raise NotImplementedError( + "Passing args to func is currently not supported." + ) + if kwargs: + raise NotImplementedError( + "Passing kwargs to func is currently not supported." + ) + + if not (isinstance(func, str) or callable(func)): raise TypeError( "Aggregation must be a named aggregation or a callable" ) try: - result = self.agg(function) + result = self.agg(func) except TypeError as e: raise NotImplementedError( "Currently, `transform()` supports only aggregations." ) from e # If the aggregation is a scan, don't broadcast - if libgroupby._is_all_scan_aggregate([[function]]): + if libgroupby._is_all_scan_aggregate([[func]]): if len(result) != len(self.obj): raise AssertionError( "Unexpected result length for scan transform" @@ -1824,7 +1889,7 @@ def func(x): return self.agg(func) @_performance_tracking - def describe(self, include=None, exclude=None): + def describe(self, percentiles=None, include=None, exclude=None): """ Generate descriptive statistics that summarizes the central tendency, dispersion and shape of a dataset's distribution, excluding NaN values. @@ -1833,6 +1898,10 @@ def describe(self, include=None, exclude=None): Parameters ---------- + percentiles : list-like of numbers, optional + The percentiles to include in the output. + Currently not supported. + include: 'all', list-like of dtypes or None (default), optional list of data types to include in the result. Ignored for Series. @@ -1869,8 +1938,12 @@ def describe(self, include=None, exclude=None): 90 1 24.0 24.0 24.0 24.0 24.0 24.0 """ - if exclude is not None and include is not None: - raise NotImplementedError + if percentiles is not None: + raise NotImplementedError("percentiles is currently not supported") + if exclude is not None: + raise NotImplementedError("exclude is currently not supported") + if include is not None: + raise NotImplementedError("include is currently not supported") res = self.agg( [ @@ -1896,69 +1969,7 @@ def describe(self, include=None, exclude=None): return res @_performance_tracking - def corr(self, method="pearson", min_periods=1): - """ - Compute pairwise correlation of columns, excluding NA/null values. - - Parameters - ---------- - method: {"pearson", "kendall", "spearman"} or callable, - default "pearson". Currently only the pearson correlation - coefficient is supported. - - min_periods: int, optional - Minimum number of observations required per pair of columns - to have a valid result. - - Returns - ------- - DataFrame - Correlation matrix. - - Examples - -------- - >>> import cudf - >>> gdf = cudf.DataFrame({ - ... "id": ["a", "a", "a", "b", "b", "b", "c", "c", "c"], - ... "val1": [5, 4, 6, 4, 8, 7, 4, 5, 2], - ... "val2": [4, 5, 6, 1, 2, 9, 8, 5, 1], - ... "val3": [4, 5, 6, 1, 2, 9, 8, 5, 1]}) - >>> gdf - id val1 val2 val3 - 0 a 5 4 4 - 1 a 4 5 5 - 2 a 6 6 6 - 3 b 4 1 1 - 4 b 8 2 2 - 5 b 7 9 9 - 6 c 4 8 8 - 7 c 5 5 5 - 8 c 2 1 1 - >>> gdf.groupby("id").corr(method="pearson") - val1 val2 val3 - id - a val1 1.000000 0.500000 0.500000 - val2 0.500000 1.000000 1.000000 - val3 0.500000 1.000000 1.000000 - b val1 1.000000 0.385727 0.385727 - val2 0.385727 1.000000 1.000000 - val3 0.385727 1.000000 1.000000 - c val1 1.000000 0.714575 0.714575 - val2 0.714575 1.000000 1.000000 - val3 0.714575 1.000000 1.000000 - """ - - if method.lower() not in ("pearson",): - raise NotImplementedError( - "Only pearson correlation is currently supported" - ) - - return self._cov_or_corr( - lambda x: x.corr(method, min_periods), "Correlation" - ) - - @_performance_tracking - def cov(self, min_periods=0, ddof=1): + def cov(self, min_periods=0, ddof=1, numeric_only: bool = False): """ Compute the pairwise covariance among the columns of a DataFrame, excluding NA/null values. @@ -2042,6 +2053,10 @@ def cov(self, min_periods=0, ddof=1): val2 3.833333 12.333333 12.333333 val3 3.833333 12.333333 12.333333 """ + if numeric_only is not False: + raise NotImplementedError( + "numeric_only is currently not supported." + ) return self._cov_or_corr( lambda x: x.cov(min_periods, ddof), "Covariance" @@ -2137,7 +2152,13 @@ def _cov_or_corr(self, func, method_name): return res @_performance_tracking - def var(self, ddof=1): + def var( + self, + ddof=1, + engine=None, + engine_kwargs=None, + numeric_only: bool = False, + ): """Compute the column-wise variance of the values in each group. Parameters @@ -2146,6 +2167,18 @@ def var(self, ddof=1): The delta degrees of freedom. N - ddof is the divisor used to normalize the variance. """ + if engine is not None: + raise NotImplementedError( + "engine is non-functional and added for compatibility with pandas" + ) + if engine_kwargs is not None: + raise NotImplementedError( + "engine_kwargs is non-functional added for compatibility with pandas" + ) + if numeric_only is not False: + raise NotImplementedError( + "numeric_only is currently not supported." + ) def func(x): return getattr(x, "var")(ddof=ddof) @@ -2153,7 +2186,13 @@ def func(x): return self.agg(func) @_performance_tracking - def std(self, ddof=1): + def std( + self, + ddof=1, + engine=None, + engine_kwargs=None, + numeric_only: bool = False, + ): """Compute the column-wise std of the values in each group. Parameters @@ -2162,6 +2201,18 @@ def std(self, ddof=1): The delta degrees of freedom. N - ddof is the divisor used to normalize the standard deviation. """ + if engine is not None: + raise NotImplementedError( + "engine is non-functional and added for compatibility with pandas" + ) + if engine_kwargs is not None: + raise NotImplementedError( + "engine_kwargs is non-functional added for compatibility with pandas" + ) + if numeric_only is not False: + raise NotImplementedError( + "numeric_only is currently not supported." + ) def func(x): return getattr(x, "std")(ddof=ddof) @@ -2169,7 +2220,9 @@ def func(x): return self.agg(func) @_performance_tracking - def quantile(self, q=0.5, interpolation="linear"): + def quantile( + self, q=0.5, interpolation="linear", numeric_only: bool = False + ): """Compute the column-wise quantiles of the values in each group. Parameters @@ -2179,7 +2232,14 @@ def quantile(self, q=0.5, interpolation="linear"): interpolation : {"linear", "lower", "higher", "midpoint", "nearest"} The interpolation method to use when the desired quantile lies between two data points. Defaults to "linear". + numeric_only : bool, default False + Include only `float`, `int` or `boolean` data. + Currently not supported """ + if numeric_only is not False: + raise NotImplementedError( + "numeric_only is not currently supported." + ) def func(x): return getattr(x, "quantile")(q=q, interpolation=interpolation) @@ -2333,7 +2393,14 @@ def fillna( ) @_performance_tracking - def shift(self, periods=1, freq=None, axis=0, fill_value=None): + def shift( + self, + periods=1, + freq=None, + axis=0, + fill_value=None, + suffix: str | None = None, + ): """ Shift each group by ``periods`` positions. @@ -2355,6 +2422,10 @@ def shift(self, periods=1, freq=None, axis=0, fill_value=None): the list. The length of the list should match the number of columns shifted. Each value should match the data type of the column to fill. + suffix : str, optional + A string to add to each shifted column if there are multiple periods. + Ignored otherwise. + Currently not supported. Returns ------- @@ -2374,6 +2445,9 @@ def shift(self, periods=1, freq=None, axis=0, fill_value=None): if not axis == 0: raise NotImplementedError("Only axis=0 is supported.") + if suffix is not None: + raise NotImplementedError("shift is not currently supported.") + values = self.grouping.values if is_list_like(fill_value): if len(fill_value) != len(values._data): @@ -2473,6 +2547,142 @@ def pct_change( shifted = fill_grp.shift(periods=periods, freq=freq) return (filled / shifted) - 1 + def _mimic_pandas_order( + self, result: DataFrameOrSeries + ) -> DataFrameOrSeries: + """Given a groupby result from libcudf, reconstruct the row orders + matching that of pandas. This also adds appropriate indices. + """ + # TODO: copy metadata after this method is a common pattern, should + # merge in this method. + + # This function is used to reorder the results of scan-based + # groupbys which have the same output size as input size. + # However, if the grouping key has NAs and dropna=True, the + # result coming back from libcudf has null_count few rows than + # the input, so we must produce an ordering from the full + # input range. + _, _, (ordering,) = self._groupby.groups( + [as_column(range(0, len(self.obj)))] + ) + if self._dropna and any( + c.has_nulls(include_nan=True) > 0 + for c in self.grouping._key_columns + ): + # Scan aggregations with null/nan keys put nulls in the + # corresponding output rows in pandas, to do that here + # expand the result by reindexing. + ri = cudf.RangeIndex(0, len(self.obj)) + result.index = cudf.Index(ordering) + # This reorders and expands + result = result.reindex(ri) + else: + # Just reorder according to the groupings + result = result.take(ordering.argsort()) + # Now produce the actual index we first thought of + result.index = self.obj.index + return result + + def ohlc(self): + """ + Compute open, high, low and close values of a group, excluding missing values. + + Currently not implemented. + """ + raise NotImplementedError("ohlc is currently not implemented") + + @property + def plot(self): + """ + Make plots of a grouped Series or DataFrame. + + Currently not implemented. + """ + raise NotImplementedError("plot is currently not implemented") + + def resample(self, rule, *args, include_groups: bool = True, **kwargs): + """ + Provide resampling when using a TimeGrouper. + + Currently not implemented. + """ + raise NotImplementedError("resample is currently not implemented") + + def take(self, indices): + """ + Return the elements in the given *positional* indices in each group. + + Currently not implemented. + """ + raise NotImplementedError("take is currently not implemented") + + def filter(self, func, dropna: bool = True, *args, **kwargs): + """ + Filter elements from groups that don't satisfy a criterion. + + Currently not implemented. + """ + raise NotImplementedError("filter is currently not implemented") + + def expanding(self, *args, **kwargs): + """ + Return an expanding grouper, providing expanding + functionality per group. + + Currently not implemented. + """ + raise NotImplementedError("expanding is currently not implemented") + + def ewm(self, *args, **kwargs): + """ + Return an ewm grouper, providing ewm functionality per group. + + Currently not implemented. + """ + raise NotImplementedError("expanding is currently not implemented") + + def any(self, skipna: bool = True): + """ + Return True if any value in the group is truthful, else False. + + Currently not implemented. + """ + raise NotImplementedError("any is currently not implemented") + + def all(self, skipna: bool = True): + """ + Return True if all values in the group are truthful, else False. + + Currently not implemented. + """ + raise NotImplementedError("all is currently not implemented") + + +class DataFrameGroupBy(GroupBy, GetAttrGetItemMixin): + obj: "cudf.core.dataframe.DataFrame" + + _PROTECTED_KEYS = frozenset(("obj",)) + + def _reduce_numeric_only(self, op: str): + columns = list( + name + for name in self.obj._data.names + if ( + is_numeric_dtype(self.obj._data[name].dtype) + and name not in self.grouping.names + ) + ) + return self[columns].agg(op) + + def __getitem__(self, key): + return self.obj[key].groupby( + by=self.grouping.keys, + dropna=self._dropna, + sort=self._sort, + group_keys=self._group_keys, + as_index=self._as_index, + ) + def value_counts( self, subset=None, @@ -2637,68 +2847,112 @@ def value_counts( return result - def _mimic_pandas_order( - self, result: DataFrameOrSeries - ) -> DataFrameOrSeries: - """Given a groupby result from libcudf, reconstruct the row orders - matching that of pandas. This also adds appropriate indices. + @_performance_tracking + def corr( + self, method="pearson", min_periods=1, numeric_only: bool = False + ): """ - # TODO: copy metadata after this method is a common pattern, should - # merge in this method. + Compute pairwise correlation of columns, excluding NA/null values. - # This function is used to reorder the results of scan-based - # groupbys which have the same output size as input size. - # However, if the grouping key has NAs and dropna=True, the - # result coming back from libcudf has null_count few rows than - # the input, so we must produce an ordering from the full - # input range. - _, _, (ordering,) = self._groupby.groups( - [as_column(range(0, len(self.obj)))] - ) - if self._dropna and any( - c.has_nulls(include_nan=True) > 0 - for c in self.grouping._key_columns - ): - # Scan aggregations with null/nan keys put nulls in the - # corresponding output rows in pandas, to do that here - # expand the result by reindexing. - ri = cudf.RangeIndex(0, len(self.obj)) - result.index = cudf.Index(ordering) - # This reorders and expands - result = result.reindex(ri) - else: - # Just reorder according to the groupings - result = result.take(ordering.argsort()) - # Now produce the actual index we first thought of - result.index = self.obj.index - return result + Parameters + ---------- + method: {"pearson", "kendall", "spearman"} or callable, + default "pearson". Currently only the pearson correlation + coefficient is supported. + min_periods: int, optional + Minimum number of observations required per pair of columns + to have a valid result. -class DataFrameGroupBy(GroupBy, GetAttrGetItemMixin): - obj: "cudf.core.dataframe.DataFrame" + Returns + ------- + DataFrame + Correlation matrix. - _PROTECTED_KEYS = frozenset(("obj",)) + Examples + -------- + >>> import cudf + >>> gdf = cudf.DataFrame({ + ... "id": ["a", "a", "a", "b", "b", "b", "c", "c", "c"], + ... "val1": [5, 4, 6, 4, 8, 7, 4, 5, 2], + ... "val2": [4, 5, 6, 1, 2, 9, 8, 5, 1], + ... "val3": [4, 5, 6, 1, 2, 9, 8, 5, 1]}) + >>> gdf + id val1 val2 val3 + 0 a 5 4 4 + 1 a 4 5 5 + 2 a 6 6 6 + 3 b 4 1 1 + 4 b 8 2 2 + 5 b 7 9 9 + 6 c 4 8 8 + 7 c 5 5 5 + 8 c 2 1 1 + >>> gdf.groupby("id").corr(method="pearson") + val1 val2 val3 + id + a val1 1.000000 0.500000 0.500000 + val2 0.500000 1.000000 1.000000 + val3 0.500000 1.000000 1.000000 + b val1 1.000000 0.385727 0.385727 + val2 0.385727 1.000000 1.000000 + val3 0.385727 1.000000 1.000000 + c val1 1.000000 0.714575 0.714575 + val2 0.714575 1.000000 1.000000 + val3 0.714575 1.000000 1.000000 + """ - def _reduce_numeric_only(self, op: str): - columns = list( - name - for name in self.obj._data.names - if ( - is_numeric_dtype(self.obj._data[name].dtype) - and name not in self.grouping.names + if method != "pearson": + raise NotImplementedError( + "Only pearson correlation is currently supported" + ) + if numeric_only is not False: + raise NotImplementedError( + "numeric_only is currently not supported." ) - ) - return self[columns].agg(op) - def __getitem__(self, key): - return self.obj[key].groupby( - by=self.grouping.keys, - dropna=self._dropna, - sort=self._sort, - group_keys=self._group_keys, - as_index=self._as_index, + return self._cov_or_corr( + lambda x: x.corr(method, min_periods), "Correlation" ) + def hist( + self, + column=None, + by=None, + grid: bool = True, + xlabelsize: int | None = None, + xrot: float | None = None, + ylabelsize: int | None = None, + yrot: float | None = None, + ax=None, + sharex: bool = False, + sharey: bool = False, + figsize: tuple[float, float] | None = None, + layout: tuple[int, int] | None = None, + bins: int | abc.Sequence[int] = 10, + backend: str | None = None, + legend: bool = False, + **kwargs, + ): + raise NotImplementedError("hist is not currently implemented") + + def boxplot( + self, + subplots: bool = True, + column=None, + fontsize: int | None = None, + rot: int = 0, + grid: bool = True, + ax=None, + figsize: tuple[float, float] | None = None, + layout=None, + sharex: bool = False, + sharey: bool = True, + backend=None, + **kwargs, + ): + raise NotImplementedError("boxplot is not currently implemented") + DataFrameGroupBy.__doc__ = groupby_doc_template.format(ret="") @@ -2706,8 +2960,10 @@ def __getitem__(self, key): class SeriesGroupBy(GroupBy): obj: "cudf.core.series.Series" - def agg(self, func): - result = super().agg(func) + def agg(self, func, *args, engine=None, engine_kwargs=None, **kwargs): + result = super().agg( + func, *args, engine=engine, engine_kwargs=engine_kwargs, **kwargs + ) # downcast the result to a Series: if len(result._data): @@ -2722,14 +2978,95 @@ def agg(self, func): aggregate = agg - def apply(self, func, *args): - result = super().apply(func, *args) + def apply(self, func, *args, **kwargs): + result = super().apply(func, *args, **kwargs) # apply Series name to result result.name = self.obj.name return result + @property + def dtype(self) -> pd.Series: + raise NotImplementedError("dtype is currently not implemented.") + + def hist( + self, + by=None, + ax=None, + grid: bool = True, + xlabelsize: int | None = None, + xrot: float | None = None, + ylabelsize: int | None = None, + yrot: float | None = None, + figsize: tuple[float, float] | None = None, + bins: int | abc.Sequence[int] = 10, + backend: str | None = None, + legend: bool = False, + **kwargs, + ): + raise NotImplementedError("hist is currently not implemented.") + + @property + def is_monotonic_increasing(self) -> cudf.Series: + """ + Return whether each group's values are monotonically increasing. + + Currently not implemented + """ + raise NotImplementedError( + "is_monotonic_increasing is currently not implemented." + ) + + @property + def is_monotonic_decreasing(self) -> cudf.Series: + """ + Return whether each group's values are monotonically decreasing. + + Currently not implemented + """ + raise NotImplementedError( + "is_monotonic_decreasing is currently not implemented." + ) + + def nlargest( + self, n: int = 5, keep: Literal["first", "last", "all"] = "first" + ) -> cudf.Series: + """ + Return the largest n elements. + + Currently not implemented + """ + raise NotImplementedError("nlargest is currently not implemented.") + + def nsmallest( + self, n: int = 5, keep: Literal["first", "last", "all"] = "first" + ) -> cudf.Series: + """ + Return the smallest n elements. + + Currently not implemented + """ + raise NotImplementedError("nsmallest is currently not implemented.") + + def value_counts( + self, + normalize: bool = False, + sort: bool = True, + ascending: bool = False, + bins=None, + dropna: bool = True, + ) -> cudf.Series | cudf.DataFrame: + raise NotImplementedError("value_counts is currently not implemented.") + + def corr( + self, + other: cudf.Series, + method: str = "pearson", + min_periods: int | None = None, + ) -> cudf.Series: + raise NotImplementedError("corr is currently not implemented.") + SeriesGroupBy.__doc__ = groupby_doc_template.format(ret="") diff --git a/python/cudf/cudf/core/resample.py b/python/cudf/cudf/core/resample.py index 4e0c5bd86b9..715bbf89b15 100644 --- a/python/cudf/cudf/core/resample.py +++ b/python/cudf/cudf/core/resample.py @@ -43,8 +43,10 @@ def __init__(self, obj, by, axis=None, kind=None): by = _ResampleGrouping(obj, by) super().__init__(obj, by=by) - def agg(self, func): - result = super().agg(func) + def agg(self, func, *args, engine=None, engine_kwargs=None, **kwargs): + result = super().agg( + func, *args, engine=engine, engine_kwargs=engine_kwargs, **kwargs + ) if len(self.grouping.bin_labels) != len(result): index = cudf.core.index.Index( self.grouping.bin_labels, name=self.grouping.names[0] diff --git a/python/cudf/cudf/tests/test_groupby.py b/python/cudf/cudf/tests/test_groupby.py index 826a0e52f57..74f04c0584f 100644 --- a/python/cudf/cudf/tests/test_groupby.py +++ b/python/cudf/cudf/tests/test_groupby.py @@ -3885,3 +3885,28 @@ def test_group_by_raises_category_error(op): with pytest.raises(TypeError): df.groupby(df.a).agg(op) + + +def test_ngroups(): + pdf = pd.DataFrame({"a": [1, 1, 3], "b": range(3)}) + gdf = cudf.DataFrame.from_pandas(pdf) + + pgb = pdf.groupby("a") + ggb = gdf.groupby("a") + assert pgb.ngroups == ggb.ngroups + assert len(pgb) == len(ggb) + + +def test_ndim(): + pdf = pd.DataFrame({"a": [1, 1, 3], "b": range(3)}) + gdf = cudf.DataFrame.from_pandas(pdf) + + pgb = pdf.groupby("a") + ggb = gdf.groupby("a") + assert pgb.ndim == ggb.ndim + + pser = pd.Series(range(3)) + gser = cudf.Series.from_pandas(pser) + pgb = pser.groupby([0, 0, 1]) + ggb = gser.groupby(cudf.Series([0, 0, 1])) + assert pgb.ndim == ggb.ndim