diff --git a/dask/dataframe/_compat.py b/dask/dataframe/_compat.py index 150f475ad84..7b8691ee4f5 100644 --- a/dask/dataframe/_compat.py +++ b/dask/dataframe/_compat.py @@ -1,4 +1,6 @@ +import contextlib import string +import warnings import numpy as np import pandas as pd @@ -82,3 +84,18 @@ def makeMixedDataFrame(): } ) return df + + +@contextlib.contextmanager +def check_numeric_only_deprecation(): + + if PANDAS_GT_150: + with warnings.catch_warnings(): + warnings.filterwarnings( + "ignore", + message="The default value of numeric_only in", + category=FutureWarning, + ) + yield + else: + yield diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 31c99380108..cc82fd791b6 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -33,7 +33,11 @@ from dask.blockwise import Blockwise, BlockwiseDep, BlockwiseDepDict, blockwise from dask.context import globalmethod from dask.dataframe import methods -from dask.dataframe._compat import PANDAS_GT_140, PANDAS_GT_150 +from dask.dataframe._compat import ( + PANDAS_GT_140, + PANDAS_GT_150, + check_numeric_only_deprecation, +) from dask.dataframe.accessor import CachedAccessor, DatetimeAccessor, StringAccessor from dask.dataframe.categorical import CategoricalAccessor, categorize from dask.dataframe.dispatch import ( @@ -1886,26 +1890,49 @@ def shift(self, periods=1, freq=None, axis=0): ) return maybe_shift_divisions(out, periods, freq=freq) - def _reduction_agg(self, name, axis=None, skipna=True, split_every=False, out=None): + def _reduction_agg( + self, + name, + axis=None, + skipna=True, + split_every=False, + out=None, + numeric_only=None, + ): axis = self._validate_axis(axis) - meta = getattr(self._meta_nonempty, name)(axis=axis, skipna=skipna) - token = self._token_prefix + name + if has_keyword(getattr(self._meta_nonempty, name), "numeric_only"): + numeric_only_kwargs = {"numeric_only": numeric_only} + else: + numeric_only_kwargs = {} - method = getattr(M, name) + with check_numeric_only_deprecation(): + meta = getattr(self._meta_nonempty, name)( + axis=axis, skipna=skipna, **numeric_only_kwargs + ) + + token = self._token_prefix + name if axis == 1: result = self.map_partitions( - method, meta=meta, token=token, skipna=skipna, axis=axis + _getattr_numeric_only, + meta=meta, + token=token, + skipna=skipna, + axis=axis, + _dask_method_name=name, + **numeric_only_kwargs, ) return handle_out(out, result) else: result = self.reduction( - method, + _getattr_numeric_only, meta=meta, token=token, skipna=skipna, axis=axis, split_every=split_every, + _dask_method_name=name, + **numeric_only_kwargs, ) if isinstance(self, DataFrame): result.divisions = (self.columns.min(), self.columns.max()) @@ -1982,7 +2009,11 @@ def prod( numeric_only=None, ): result = self._reduction_agg( - "prod", axis=axis, skipna=skipna, split_every=split_every, out=out + "prod", + axis=axis, + skipna=skipna, + split_every=split_every, + out=out, ) if min_count: cond = self.notnull().sum(axis=axis) >= min_count @@ -2003,7 +2034,11 @@ def max( self, axis=None, skipna=True, split_every=False, out=None, numeric_only=None ): return self._reduction_agg( - "max", axis=axis, skipna=skipna, split_every=split_every, out=out + "max", + axis=axis, + skipna=skipna, + split_every=split_every, + out=out, ) @_numeric_only @@ -2012,7 +2047,11 @@ def min( self, axis=None, skipna=True, split_every=False, out=None, numeric_only=None ): return self._reduction_agg( - "min", axis=axis, skipna=skipna, split_every=split_every, out=out + "min", + axis=axis, + skipna=skipna, + split_every=split_every, + out=out, ) @derived_from(pd.DataFrame) @@ -2132,7 +2171,11 @@ def mean( ): axis = self._validate_axis(axis) _raise_if_object_series(self, "mean") - meta = self._meta_nonempty.mean(axis=axis, skipna=skipna) + # NOTE: Do we want to warn here? + with check_numeric_only_deprecation(): + meta = self._meta_nonempty.mean( + axis=axis, skipna=skipna, numeric_only=numeric_only + ) if axis == 1: result = map_partitions( M.mean, @@ -2142,6 +2185,7 @@ def mean( axis=axis, skipna=skipna, enforce_metadata=False, + numeric_only=numeric_only, ) return handle_out(out, result) else: @@ -2204,7 +2248,10 @@ def var( ): axis = self._validate_axis(axis) _raise_if_object_series(self, "var") - meta = self._meta_nonempty.var(axis=axis, skipna=skipna) + with check_numeric_only_deprecation(): + meta = self._meta_nonempty.var( + axis=axis, skipna=skipna, numeric_only=numeric_only + ) if axis == 1: result = map_partitions( M.var, @@ -2215,6 +2262,7 @@ def var( skipna=skipna, ddof=ddof, enforce_metadata=False, + numeric_only=numeric_only, ) return handle_out(out, result) else: @@ -2351,7 +2399,10 @@ def std( _raise_if_object_series(self, "std") _raise_if_not_series_or_dataframe(self, "std") - meta = self._meta_nonempty.std(axis=axis, skipna=skipna) + with check_numeric_only_deprecation(): + meta = self._meta_nonempty.std( + axis=axis, skipna=skipna, numeric_only=numeric_only + ) is_df_like = is_dataframe_like(self._meta) needs_time_conversion = False numeric_dd = self @@ -2378,6 +2429,7 @@ def std( skipna=skipna, ddof=ddof, enforce_metadata=False, + numeric_only=numeric_only, parent_meta=self._meta, ) return handle_out(out, result) @@ -2671,7 +2723,10 @@ def _kurtosis_numeric(self, fisher=True, bias=True, nan_policy="propagate"): def sem(self, axis=None, skipna=True, ddof=1, split_every=False, numeric_only=None): axis = self._validate_axis(axis) _raise_if_object_series(self, "sem") - meta = self._meta_nonempty.sem(axis=axis, skipna=skipna, ddof=ddof) + with check_numeric_only_deprecation(): + meta = self._meta_nonempty.sem( + axis=axis, skipna=skipna, ddof=ddof, numeric_only=numeric_only + ) if axis == 1: return map_partitions( M.sem, @@ -2682,6 +2737,7 @@ def sem(self, axis=None, skipna=True, ddof=1, split_every=False, numeric_only=No skipna=skipna, ddof=ddof, parent_meta=self._meta, + numeric_only=numeric_only, ) else: num = self._get_numeric_data() @@ -2701,7 +2757,8 @@ def sem(self, axis=None, skipna=True, ddof=1, split_every=False, numeric_only=No result.divisions = (self.columns.min(), self.columns.max()) return result - def quantile(self, q=0.5, axis=0, method="default"): + @_numeric_only + def quantile(self, q=0.5, axis=0, numeric_only=True, method="default"): """Approximate row-wise and precise column-wise quantiles of DataFrame Parameters @@ -2717,11 +2774,13 @@ def quantile(self, q=0.5, axis=0, method="default"): """ axis = self._validate_axis(axis) keyname = "quantiles-concat--" + tokenize(self, q, axis) + meta = self._meta.quantile(q, axis=axis, numeric_only=numeric_only) if axis == 1: if isinstance(q, list): # Not supported, the result will have current index as columns raise ValueError("'q' must be scalar when axis=1 is specified") + return map_partitions( M.quantile, self, @@ -2729,12 +2788,12 @@ def quantile(self, q=0.5, axis=0, method="default"): axis, token=keyname, enforce_metadata=False, + numeric_only=numeric_only, meta=(q, "f8"), parent_meta=self._meta, ) else: _raise_if_object_series(self, "quantile") - meta = self._meta.quantile(q, axis=axis) num = self._get_numeric_data() quantiles = tuple(quantile(self[c], q, method) for c in num.columns) @@ -6519,7 +6578,7 @@ def _emulate(func, *args, udf=False, **kwargs): Apply a function using args / kwargs. If arguments contain dd.DataFrame / dd.Series, using internal cache (``_meta``) for calculation """ - with raise_on_meta_error(funcname(func), udf=udf): + with raise_on_meta_error(funcname(func), udf=udf), check_numeric_only_deprecation(): return func(*_extract_meta(args, True), **_extract_meta(kwargs, True)) @@ -8123,3 +8182,8 @@ def _raise_if_not_series_or_dataframe(x, funcname): "`%s` is only supported with objects that are Dataframes or Series" % funcname ) + + +def _getattr_numeric_only(*args, _dask_method_name, **kwargs): + with check_numeric_only_deprecation(): + return getattr(M, _dask_method_name)(*args, **kwargs) diff --git a/dask/dataframe/groupby.py b/dask/dataframe/groupby.py index 3415008db73..845be09611d 100644 --- a/dask/dataframe/groupby.py +++ b/dask/dataframe/groupby.py @@ -10,7 +10,7 @@ from dask import config from dask.base import tokenize -from dask.dataframe._compat import PANDAS_GT_150 +from dask.dataframe._compat import PANDAS_GT_150, check_numeric_only_deprecation from dask.dataframe.core import ( GROUP_KEYS_DEFAULT, DataFrame, @@ -346,7 +346,8 @@ def _var_chunk(df, *by): df = df.copy() g = _groupby_raise_unaligned(df, by=by) - x = g.sum() + with check_numeric_only_deprecation(): + x = g.sum() n = g[x.columns].count().rename(columns=lambda c: (c, "-count")) @@ -354,7 +355,8 @@ def _var_chunk(df, *by): df[cols] = df[cols] ** 2 g2 = _groupby_raise_unaligned(df, by=by) - x2 = g2.sum().rename(columns=lambda c: (c, "-x2")) + with check_numeric_only_deprecation(): + x2 = g2.sum().rename(columns=lambda c: (c, "-x2")) return concat([x, x2, n], axis=1) @@ -1251,7 +1253,8 @@ def _aca_agg( aggfunc = func if meta is None: - meta = func(self._meta_nonempty) + with check_numeric_only_deprecation(): + meta = func(self._meta_nonempty) if chunk_kwargs is None: chunk_kwargs = {} diff --git a/dask/dataframe/tests/test_arithmetics_reduction.py b/dask/dataframe/tests/test_arithmetics_reduction.py index f73ce1e2cc9..bd25db03cad 100644 --- a/dask/dataframe/tests/test_arithmetics_reduction.py +++ b/dask/dataframe/tests/test_arithmetics_reduction.py @@ -7,7 +7,11 @@ from pandas.api.types import is_scalar import dask.dataframe as dd -from dask.dataframe._compat import PANDAS_GT_120, PANDAS_VERSION +from dask.dataframe._compat import ( + PANDAS_GT_120, + PANDAS_VERSION, + check_numeric_only_deprecation, +) from dask.dataframe.utils import assert_dask_graph, assert_eq, make_meta try: @@ -1161,27 +1165,51 @@ def test_reductions_frame_dtypes(): ddf_no_timedelta = dd.from_pandas(df_no_timedelta, 3) assert_eq(df.drop(columns="dt").sum(), ddf.drop(columns="dt").sum()) + with check_numeric_only_deprecation(): + expected = df_no_timedelta.drop(columns="dt").mean() assert_eq( - df_no_timedelta.drop(columns="dt").mean(), + expected, ddf_no_timedelta.drop(columns="dt").mean(), ) - assert_eq(df.prod(), ddf.prod()) - assert_eq(df.product(), ddf.product()) + with check_numeric_only_deprecation(): + expected = df.prod() + assert_eq(expected, ddf.prod()) + with check_numeric_only_deprecation(): + expected = df.product() + assert_eq(expected, ddf.product()) assert_eq(df.min(), ddf.min()) assert_eq(df.max(), ddf.max()) assert_eq(df.count(), ddf.count()) - assert_eq(df.sem(), ddf.sem()) - assert_eq(df.sem(ddof=0), ddf.sem(ddof=0)) - - assert_eq(df_no_timedelta.std(), ddf_no_timedelta.std()) - assert_eq(df_no_timedelta.std(skipna=False), ddf_no_timedelta.std(skipna=False)) - assert_eq(df_no_timedelta.std(ddof=0), ddf_no_timedelta.std(ddof=0)) - assert_eq(df_no_timedelta.var(), ddf_no_timedelta.var()) - assert_eq(df_no_timedelta.var(skipna=False), ddf_no_timedelta.var(skipna=False)) - assert_eq(df_no_timedelta.var(ddof=0), ddf_no_timedelta.var(ddof=0)) + with check_numeric_only_deprecation(): + expected = df.sem() + assert_eq(expected, ddf.sem()) + with check_numeric_only_deprecation(): + expected = df.sem(ddof=0) + assert_eq(expected, ddf.sem(ddof=0)) + + with check_numeric_only_deprecation(): + expected = df_no_timedelta.std() + assert_eq(expected, ddf_no_timedelta.std()) + with check_numeric_only_deprecation(): + expected = df_no_timedelta.std(skipna=False) + assert_eq(expected, ddf_no_timedelta.std(skipna=False)) + with check_numeric_only_deprecation(): + expected = df_no_timedelta.std(ddof=0) + assert_eq(expected, ddf_no_timedelta.std(ddof=0)) + with check_numeric_only_deprecation(): + expected = df_no_timedelta.var() + assert_eq(expected, ddf_no_timedelta.var()) + with check_numeric_only_deprecation(): + expected = df_no_timedelta.var(skipna=False) + assert_eq(expected, ddf_no_timedelta.var(skipna=False)) + with check_numeric_only_deprecation(): + expected = df_no_timedelta.var(ddof=0) + assert_eq(expected, ddf_no_timedelta.var(ddof=0)) + with check_numeric_only_deprecation(): + expected = df_no_timedelta.var(ddof=0, skipna=False) assert_eq( - df_no_timedelta.var(ddof=0, skipna=False), + expected, ddf_no_timedelta.var(ddof=0, skipna=False), ) @@ -1195,8 +1223,12 @@ def test_reductions_frame_dtypes(): # only timedelta df_td = df[["timedelta"]] ddf_td = dd.from_pandas(df_td, 3) - assert_eq(df_td.var(ddof=0), ddf_td.var(ddof=0)) - assert_eq(df_td.var(), ddf_td.var()) + with check_numeric_only_deprecation(): + expected = df_td.var(ddof=0) + assert_eq(expected, ddf_td.var(ddof=0)) + with check_numeric_only_deprecation(): + expected = df_td.var() + assert_eq(expected, ddf_td.var()) # only numercis df_numerics = df[["int", "float", "bool"]] diff --git a/dask/dataframe/tests/test_categorical.py b/dask/dataframe/tests/test_categorical.py index 8dbb3d2a864..160f734a3c3 100644 --- a/dask/dataframe/tests/test_categorical.py +++ b/dask/dataframe/tests/test_categorical.py @@ -8,7 +8,7 @@ import dask import dask.dataframe as dd from dask.dataframe import _compat -from dask.dataframe._compat import tm +from dask.dataframe._compat import check_numeric_only_deprecation, tm from dask.dataframe.core import _concat from dask.dataframe.utils import ( assert_eq, @@ -133,7 +133,9 @@ def test_unknown_categoricals(shuffle_method): assert_eq(ddf.w.value_counts(), df.w.value_counts()) assert_eq(ddf.w.nunique(), df.w.nunique()) - assert_eq(ddf.groupby(ddf.w).sum(), df.groupby(df.w).sum()) + with check_numeric_only_deprecation(): + expected = df.groupby(df.w).sum() + assert_eq(ddf.groupby(ddf.w).sum(), expected) assert_eq(ddf.groupby(ddf.w).y.nunique(), df.groupby(df.w).y.nunique()) assert_eq(ddf.y.groupby(ddf.w).count(), df.y.groupby(df.w).count()) diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index e66a1f168b5..8ce96e750a4 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -24,6 +24,7 @@ PANDAS_GT_120, PANDAS_GT_140, PANDAS_GT_150, + check_numeric_only_deprecation, tm, ) from dask.dataframe.core import ( @@ -1511,7 +1512,8 @@ def test_dataframe_quantile(method, expected): ) ddf = dd.from_pandas(df, 3) - result = ddf.quantile(method=method) + with check_numeric_only_deprecation(): + result = ddf.quantile(method=method) assert result.npartitions == 1 assert result.divisions == ("A", "X") @@ -1532,7 +1534,9 @@ def test_dataframe_quantile(method, expected): assert (result == expected[1]).all().all() - assert_eq(ddf.quantile(axis=1, method=method), df.quantile(axis=1)) + with check_numeric_only_deprecation(): + expected = df.quantile(axis=1) + assert_eq(ddf.quantile(axis=1, method=method), expected) pytest.raises(ValueError, lambda: ddf.quantile([0.25, 0.75], axis=1, method=method)) @@ -3306,8 +3310,12 @@ def test_cov_corr_mixed(): df["unique_id"] = df["unique_id"].astype(str) ddf = dd.from_pandas(df, npartitions=20) - assert_eq(ddf.corr(split_every=4), df.corr(), check_divisions=False) - assert_eq(ddf.cov(split_every=4), df.cov(), check_divisions=False) + with check_numeric_only_deprecation(): + expected = df.corr() + assert_eq(ddf.corr(split_every=4), expected, check_divisions=False) + with check_numeric_only_deprecation(): + expected = df.cov() + assert_eq(ddf.cov(split_every=4), expected, check_divisions=False) def test_autocorr(): diff --git a/dask/dataframe/tests/test_groupby.py b/dask/dataframe/tests/test_groupby.py index 3c2d19efb54..c1de060a3d2 100644 --- a/dask/dataframe/tests/test_groupby.py +++ b/dask/dataframe/tests/test_groupby.py @@ -1,4 +1,5 @@ import collections +import contextlib import operator import pickle import warnings @@ -10,7 +11,13 @@ import dask import dask.dataframe as dd from dask.dataframe import _compat -from dask.dataframe._compat import PANDAS_GT_110, PANDAS_GT_130, PANDAS_GT_150, tm +from dask.dataframe._compat import ( + PANDAS_GT_110, + PANDAS_GT_130, + PANDAS_GT_150, + check_numeric_only_deprecation, + tm, +) from dask.dataframe.backends import grouper_dispatch from dask.dataframe.utils import assert_dask_graph, assert_eq, assert_max_deps from dask.utils import M @@ -2060,7 +2067,9 @@ def test_std_object_dtype(func): df = pd.DataFrame({"x": [1, 2, 1], "y": ["a", "b", "c"], "z": [11.0, 22.0, 33.0]}) ddf = dd.from_pandas(df, npartitions=2) - assert_eq(func(df), func(ddf)) + with check_numeric_only_deprecation(): + expected = func(df) + assert_eq(expected, func(ddf)) def test_std_columns_int(): @@ -2717,7 +2726,10 @@ def test_groupby_sort_argument(by, agg, sort): # Basic groupby aggregation result_1 = getattr(gb, agg) - result_1_pd = getattr(gb_pd, agg) + + def result_1_pd(): + with check_numeric_only_deprecation(): + return getattr(gb_pd, agg)() # Choose single column result_2 = getattr(gb.e, agg) @@ -2803,7 +2815,12 @@ def test_groupby_aggregate_categorical_observed( ddf["cat_2"] = ddf["cat_2"].cat.as_unknown() def agg(grp, **kwargs): - return getattr(grp, agg_func)(**kwargs) + if isinstance(grp, pd.core.groupby.DataFrameGroupBy): + ctx = check_numeric_only_deprecation + else: + ctx = contextlib.nullcontext + with ctx(): + return getattr(grp, agg_func)(**kwargs) # only include numeric columns when passing to "min" or "max" # pandas default is numeric_only=False