From 82d1f3f5f88655b954f5cd6db5c2d68e3abeccc9 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Mon, 12 Sep 2022 11:35:02 -0500 Subject: [PATCH 01/13] Initial checkpoint --- dask/dataframe/_compat.py | 15 +++ dask/dataframe/core.py | 110 +++++++++++++++--- .../tests/test_arithmetics_reduction.py | 86 +++++++++----- dask/dataframe/tests/test_dataframe.py | 16 ++- 4 files changed, 179 insertions(+), 48 deletions(-) diff --git a/dask/dataframe/_compat.py b/dask/dataframe/_compat.py index 150f475ad84..7c872f09c2d 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,16 @@ 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) + warnings.filterwarnings("ignore", category=FutureWarning) + + yield + else: + yield diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 184ef693802..5cea0c4450a 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 ( @@ -106,8 +110,8 @@ def wrapper(self, *args, **kwargs): raise NotImplementedError( "'numeric_only=False' is not implemented in Dask." ) - elif kwargs.get("numeric_only") is True: - self = self._get_numeric_data() + # elif kwargs.get("numeric_only") is True: + # self = self._get_numeric_data() return func(self, *args, **kwargs) return wrapper @@ -1884,27 +1888,58 @@ 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) + if has_keyword(getattr(self._meta_nonempty, name), "numeric_only"): + numeric_only_kwargs = {"numeric_only": numeric_only} + else: + numeric_only_kwargs = {} + + # TODO: maybe we should warn here? + with check_numeric_only_deprecation(): + meta = getattr(self._meta_nonempty, name)( + axis=axis, skipna=skipna, **numeric_only_kwargs + ) + token = self._token_prefix + name - method = getattr(M, name) + # method = getattr(M, name) + # def method(*args, **kwargs): + # with check_numeric_only_deprecation(): + # return getattr(M, name)(*args, **kwargs) + 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, + **numeric_only_kwargs, ) return handle_out(out, result) else: + # breakpoint() result = self.reduction( - method, + _getattr_numeric_only, meta=meta, token=token, skipna=skipna, axis=axis, split_every=split_every, + # numeric_only=Fase, + **numeric_only_kwargs, ) + # breakpoint() if isinstance(self, DataFrame): result.divisions = (self.columns.min(), self.columns.max()) return handle_out(out, result) @@ -1980,7 +2015,12 @@ 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, + numeric_only=numeric_only, ) if min_count: cond = self.notnull().sum(axis=axis) >= min_count @@ -2001,7 +2041,12 @@ 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=numeric_only, ) @_numeric_only @@ -2010,7 +2055,12 @@ 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, + numeric_only=numeric_only, ) @derived_from(pd.DataFrame) @@ -2130,7 +2180,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, @@ -2140,6 +2194,7 @@ def mean( axis=axis, skipna=skipna, enforce_metadata=False, + numeric_only=numeric_only, ) return handle_out(out, result) else: @@ -2174,7 +2229,11 @@ def var( ): axis = self._validate_axis(axis) _raise_if_object_series(self, "var") - meta = self._meta_nonempty.var(axis=axis, skipna=skipna) + # TODO: warn here + 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, @@ -2185,6 +2244,7 @@ def var( skipna=skipna, ddof=ddof, enforce_metadata=False, + numeric_only=numeric_only, ) return handle_out(out, result) else: @@ -2321,7 +2381,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 @@ -2348,6 +2411,7 @@ def std( skipna=skipna, ddof=ddof, enforce_metadata=False, + numeric_only=numeric_only, parent_meta=self._meta, ) return handle_out(out, result) @@ -2641,7 +2705,11 @@ 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) + # TODO: maybe we should raise here + 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, @@ -2652,6 +2720,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() @@ -2671,7 +2740,7 @@ 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"): + def quantile(self, q=0.5, axis=0, numeric_only=True, method="default"): """Approximate row-wise and precise column-wise quantiles of DataFrame Parameters @@ -2687,11 +2756,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, @@ -2699,12 +2770,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) @@ -8050,3 +8121,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, **kwargs): + with check_numeric_only_deprecation(): + return getattr(M, name)(*args, **kwargs) diff --git a/dask/dataframe/tests/test_arithmetics_reduction.py b/dask/dataframe/tests/test_arithmetics_reduction.py index f73ce1e2cc9..3071a30b610 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"]] @@ -1204,7 +1236,7 @@ def test_reductions_frame_dtypes(): assert_eq(df_numerics.var(), ddf_numerics.var()) -def test_reductions_frame_dtypes_numeric_only(): +def _test_reductions_frame_dtypes_numeric_only(): df = pd.DataFrame( { "int": [1, 2, 3, 4, 5, 6, 7, 8], @@ -1216,19 +1248,19 @@ def test_reductions_frame_dtypes_numeric_only(): } ) - ddf = dd.from_pandas(df, 3) + ddf = dd.from_pandas(df, 1) kwargs = {"numeric_only": True} funcs = [ - "sum", - "prod", - "product", + # "sum", + # "prod", + # "product", "min", - "max", - "mean", - "var", - "std", - "count", - "sem", + # "max", + # "mean", + # "var", + # "std", + # "count", + # "sem", ] for func in funcs: diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index 932abfa0bf1..951374076f2 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -23,6 +23,7 @@ PANDAS_GT_120, PANDAS_GT_140, PANDAS_GT_150, + check_numeric_only_deprecation, tm, ) from dask.dataframe.core import ( @@ -1456,7 +1457,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") @@ -1477,7 +1479,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)) @@ -3243,8 +3247,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(): From cdd501d8b851a7e3fbea35d06b3224463b9a058e Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 14 Sep 2022 11:32:04 -0500 Subject: [PATCH 02/13] test-upstream From 4721b4f288c510aec65f5925f1306d0b1699027f Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 14 Sep 2022 12:08:00 -0500 Subject: [PATCH 03/13] Pass method name [test-upstream] --- dask/dataframe/core.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 5cea0c4450a..f83e009e958 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -1924,6 +1924,7 @@ def _reduction_agg( token=token, skipna=skipna, axis=axis, + _name=name, **numeric_only_kwargs, ) return handle_out(out, result) @@ -1936,6 +1937,7 @@ def _reduction_agg( skipna=skipna, axis=axis, split_every=split_every, + _name=name, # numeric_only=Fase, **numeric_only_kwargs, ) @@ -8124,5 +8126,6 @@ def _raise_if_not_series_or_dataframe(x, funcname): def _getattr_numeric_only(*args, **kwargs): + name = kwargs.pop("_name") with check_numeric_only_deprecation(): return getattr(M, name)(*args, **kwargs) From 85ce4c63fdb01556891f8bfd757dab81885d6785 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 14 Sep 2022 13:05:16 -0500 Subject: [PATCH 04/13] Groupby [test-upstream] --- dask/dataframe/core.py | 2 +- dask/dataframe/groupby.py | 11 +++++++---- dask/dataframe/tests/test_groupby.py | 25 +++++++++++++++++++++---- 3 files changed, 29 insertions(+), 9 deletions(-) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index f83e009e958..e18ba3df9a3 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -6519,7 +6519,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)) diff --git a/dask/dataframe/groupby.py b/dask/dataframe/groupby.py index de110f7c5ac..f858a96cdf4 100644 --- a/dask/dataframe/groupby.py +++ b/dask/dataframe/groupby.py @@ -9,7 +9,7 @@ import pandas as pd 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, @@ -345,7 +345,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")) @@ -353,7 +354,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) @@ -1245,7 +1247,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_groupby.py b/dask/dataframe/tests/test_groupby.py index 79e343bb968..2ee7a36604d 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 @@ -2032,7 +2039,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(): @@ -2689,7 +2698,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) @@ -2775,7 +2787,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 From 61360616b7ab564b2f8a6db0bba6b36ee0063f69 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 14 Sep 2022 14:14:50 -0500 Subject: [PATCH 05/13] Cleanup [test-upstream] --- dask/dataframe/core.py | 14 ++----------- .../tests/test_arithmetics_reduction.py | 20 +++++++++---------- 2 files changed, 12 insertions(+), 22 deletions(-) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index e18ba3df9a3..cc093c4a979 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -110,8 +110,8 @@ def wrapper(self, *args, **kwargs): raise NotImplementedError( "'numeric_only=False' is not implemented in Dask." ) - # elif kwargs.get("numeric_only") is True: - # self = self._get_numeric_data() + elif kwargs.get("numeric_only") is True: + self = self._get_numeric_data() return func(self, *args, **kwargs) return wrapper @@ -1911,12 +1911,6 @@ def _reduction_agg( ) token = self._token_prefix + name - - # method = getattr(M, name) - # def method(*args, **kwargs): - # with check_numeric_only_deprecation(): - # return getattr(M, name)(*args, **kwargs) - if axis == 1: result = self.map_partitions( _getattr_numeric_only, @@ -1929,7 +1923,6 @@ def _reduction_agg( ) return handle_out(out, result) else: - # breakpoint() result = self.reduction( _getattr_numeric_only, meta=meta, @@ -1938,7 +1931,6 @@ def _reduction_agg( axis=axis, split_every=split_every, _name=name, - # numeric_only=Fase, **numeric_only_kwargs, ) # breakpoint() @@ -2048,7 +2040,6 @@ def max( skipna=skipna, split_every=split_every, out=out, - numeric_only=numeric_only, ) @_numeric_only @@ -2062,7 +2053,6 @@ def min( skipna=skipna, split_every=split_every, out=out, - numeric_only=numeric_only, ) @derived_from(pd.DataFrame) diff --git a/dask/dataframe/tests/test_arithmetics_reduction.py b/dask/dataframe/tests/test_arithmetics_reduction.py index 3071a30b610..ddbc4fb1cf4 100644 --- a/dask/dataframe/tests/test_arithmetics_reduction.py +++ b/dask/dataframe/tests/test_arithmetics_reduction.py @@ -1236,7 +1236,7 @@ def test_reductions_frame_dtypes(): assert_eq(df_numerics.var(), ddf_numerics.var()) -def _test_reductions_frame_dtypes_numeric_only(): +def test_reductions_frame_dtypes_numeric_only(): df = pd.DataFrame( { "int": [1, 2, 3, 4, 5, 6, 7, 8], @@ -1251,16 +1251,16 @@ def _test_reductions_frame_dtypes_numeric_only(): ddf = dd.from_pandas(df, 1) kwargs = {"numeric_only": True} funcs = [ - # "sum", - # "prod", - # "product", + "sum", + "prod", + "product", "min", - # "max", - # "mean", - # "var", - # "std", - # "count", - # "sem", + "max", + "mean", + "var", + "std", + "count", + "sem", ] for func in funcs: From 080a176b3452385e88d579c5cd6cf891bcea31c7 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 14 Sep 2022 14:29:56 -0500 Subject: [PATCH 06/13] More specific warning catching [test-upstream] --- dask/dataframe/_compat.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dask/dataframe/_compat.py b/dask/dataframe/_compat.py index 7c872f09c2d..7b8691ee4f5 100644 --- a/dask/dataframe/_compat.py +++ b/dask/dataframe/_compat.py @@ -91,9 +91,11 @@ 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) - warnings.filterwarnings("ignore", category=FutureWarning) - + warnings.filterwarnings( + "ignore", + message="The default value of numeric_only in", + category=FutureWarning, + ) yield else: yield From d356bda64d42f05fd14ec08d4f2b64ea3c38a267 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 14 Sep 2022 14:36:28 -0500 Subject: [PATCH 07/13] Remove stray breakpoint [test-upstream] --- dask/dataframe/core.py | 1 - 1 file changed, 1 deletion(-) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index cc093c4a979..50152ed33f2 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -1933,7 +1933,6 @@ def _reduction_agg( _name=name, **numeric_only_kwargs, ) - # breakpoint() if isinstance(self, DataFrame): result.divisions = (self.columns.min(), self.columns.max()) return handle_out(out, result) From 1353a503fc9646dd5f40177782d4e2ce094249c6 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 14 Sep 2022 16:17:11 -0500 Subject: [PATCH 08/13] Fix categorical tests [test-upstream] --- dask/dataframe/tests/test_categorical.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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()) From 29bfc4819351319024895b96349740fb1874391e Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Thu, 15 Sep 2022 10:25:01 -0500 Subject: [PATCH 09/13] Restore npartitions after debugging [test-upstream] --- dask/dataframe/tests/test_arithmetics_reduction.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask/dataframe/tests/test_arithmetics_reduction.py b/dask/dataframe/tests/test_arithmetics_reduction.py index ddbc4fb1cf4..bd25db03cad 100644 --- a/dask/dataframe/tests/test_arithmetics_reduction.py +++ b/dask/dataframe/tests/test_arithmetics_reduction.py @@ -1248,7 +1248,7 @@ def test_reductions_frame_dtypes_numeric_only(): } ) - ddf = dd.from_pandas(df, 1) + ddf = dd.from_pandas(df, 3) kwargs = {"numeric_only": True} funcs = [ "sum", From 507bcf0d029905e969460fd85e55c0aab75bc011 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Thu, 15 Sep 2022 12:24:29 -0500 Subject: [PATCH 10/13] Updates [test-upstream] --- dask/dataframe/core.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 50152ed33f2..7f0226fff72 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -2013,7 +2013,6 @@ def prod( skipna=skipna, split_every=split_every, out=out, - numeric_only=numeric_only, ) if min_count: cond = self.notnull().sum(axis=axis) >= min_count @@ -2731,6 +2730,7 @@ 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 + @_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 From 209dec808c341e7e27d777fb71cd036168f080a2 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Thu, 15 Sep 2022 12:54:04 -0500 Subject: [PATCH 11/13] Roll back columns [test-upstream] --- dask/dataframe/core.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 7f0226fff72..b03cdca4e2a 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -1904,7 +1904,6 @@ def _reduction_agg( else: numeric_only_kwargs = {} - # TODO: maybe we should warn here? with check_numeric_only_deprecation(): meta = getattr(self._meta_nonempty, name)( axis=axis, skipna=skipna, **numeric_only_kwargs @@ -2219,7 +2218,6 @@ def var( ): axis = self._validate_axis(axis) _raise_if_object_series(self, "var") - # TODO: warn here with check_numeric_only_deprecation(): meta = self._meta_nonempty.var( axis=axis, skipna=skipna, numeric_only=numeric_only @@ -2695,7 +2693,6 @@ 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") - # TODO: maybe we should raise here with check_numeric_only_deprecation(): meta = self._meta_nonempty.sem( axis=axis, skipna=skipna, ddof=ddof, numeric_only=numeric_only From 3a79096f5a5fbb6273d67d44b6e2c44918c7208b Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Thu, 15 Sep 2022 16:24:02 -0500 Subject: [PATCH 12/13] Be more explicit about method name in _getattr_numeric_only [test-upstream] --- dask/dataframe/core.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index b03cdca4e2a..2e45c283239 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -1917,7 +1917,7 @@ def _reduction_agg( token=token, skipna=skipna, axis=axis, - _name=name, + method_name=name, **numeric_only_kwargs, ) return handle_out(out, result) @@ -1929,7 +1929,7 @@ def _reduction_agg( skipna=skipna, axis=axis, split_every=split_every, - _name=name, + method_name=name, **numeric_only_kwargs, ) if isinstance(self, DataFrame): @@ -8111,7 +8111,6 @@ def _raise_if_not_series_or_dataframe(x, funcname): ) -def _getattr_numeric_only(*args, **kwargs): - name = kwargs.pop("_name") +def _getattr_numeric_only(*args, method_name, **kwargs): with check_numeric_only_deprecation(): - return getattr(M, name)(*args, **kwargs) + return getattr(M, method_name)(*args, **kwargs) From 5e3efcd6f61980d933d49dc55d3e01163e145e0e Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Thu, 15 Sep 2022 16:56:41 -0500 Subject: [PATCH 13/13] Use more specific parameter for method name [test-upstream] --- dask/dataframe/core.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 2e45c283239..98ea0d3fd1c 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -1917,7 +1917,7 @@ def _reduction_agg( token=token, skipna=skipna, axis=axis, - method_name=name, + _dask_method_name=name, **numeric_only_kwargs, ) return handle_out(out, result) @@ -1929,7 +1929,7 @@ def _reduction_agg( skipna=skipna, axis=axis, split_every=split_every, - method_name=name, + _dask_method_name=name, **numeric_only_kwargs, ) if isinstance(self, DataFrame): @@ -8111,6 +8111,6 @@ def _raise_if_not_series_or_dataframe(x, funcname): ) -def _getattr_numeric_only(*args, method_name, **kwargs): +def _getattr_numeric_only(*args, _dask_method_name, **kwargs): with check_numeric_only_deprecation(): - return getattr(M, method_name)(*args, **kwargs) + return getattr(M, _dask_method_name)(*args, **kwargs)