Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Filter out numeric_only warnings from pandas #9496

Merged
merged 13 commits into from Sep 15, 2022
17 changes: 17 additions & 0 deletions dask/dataframe/_compat.py
@@ -1,4 +1,6 @@
import contextlib
import string
import warnings

import numpy as np
import pandas as pd
Expand Down Expand Up @@ -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
98 changes: 81 additions & 17 deletions dask/dataframe/core.py
Expand Up @@ -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 (
Expand Down Expand Up @@ -1884,26 +1888,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())
Expand Down Expand Up @@ -1980,7 +2007,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
Expand All @@ -2001,7 +2032,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,
Copy link
Member

Choose a reason for hiding this comment

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

In the min and max cases we are not passing numeric_only, because for these cases are part of " that users get a deprecation messages if (a) the defaults are used and (b) the result would change based on the new default (False) in 2.0." ? citing this comment pandas-dev/pandas#46560 (comment)

Copy link
Member Author

Choose a reason for hiding this comment

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

Locally I saw strange test failures when passing numeric_only here. Additionally, we shouldn't need to pass numeric_only through as the @_numeric_only decorator should handle selecting only numeric data for us automatically

)

@_numeric_only
Expand All @@ -2010,7 +2045,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)
Expand Down Expand Up @@ -2130,7 +2169,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,
Expand All @@ -2140,6 +2183,7 @@ def mean(
axis=axis,
skipna=skipna,
enforce_metadata=False,
numeric_only=numeric_only,
)
return handle_out(out, result)
else:
Expand Down Expand Up @@ -2174,7 +2218,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,
Expand All @@ -2185,6 +2232,7 @@ def var(
skipna=skipna,
ddof=ddof,
enforce_metadata=False,
numeric_only=numeric_only,
)
return handle_out(out, result)
else:
Expand Down Expand Up @@ -2321,7 +2369,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
Expand All @@ -2348,6 +2399,7 @@ def std(
skipna=skipna,
ddof=ddof,
enforce_metadata=False,
numeric_only=numeric_only,
Copy link
Member

Choose a reason for hiding this comment

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

This is a comment for line 2680.
kurtosis_shape = num._meta_nonempty.values.var(axis=0).shape since we are checking the meta_nonempty, don't we need the check_numeric_only_deprecation here too?

Copy link
Member Author

Choose a reason for hiding this comment

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

Since we're using .values to get a NumPy array we shouldn't trigger the warning from pandas

parent_meta=self._meta,
)
return handle_out(out, result)
Expand Down Expand Up @@ -2641,7 +2693,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,
Expand All @@ -2652,6 +2707,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()
Expand All @@ -2671,7 +2727,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"):
Copy link
Member

Choose a reason for hiding this comment

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

Looking at pandas to check for consistency. this default will be a no_default in 1.5, and it says in the future it'll be False. see https://pandas.pydata.org/docs/dev/reference/api/pandas.DataFrame.quantile.html

Do we need to warn here too?

Copy link
Member Author

Choose a reason for hiding this comment

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

Let's not warn here now as numeric_only=False isn't supported in Dask today. Let's handle this in follow-up work though

"""Approximate row-wise and precise column-wise quantiles of DataFrame

Parameters
Expand All @@ -2687,24 +2744,26 @@ 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,
q,
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)

Expand Down Expand Up @@ -6446,7 +6505,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))


Expand Down Expand Up @@ -8050,3 +8109,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)
11 changes: 7 additions & 4 deletions dask/dataframe/groupby.py
Expand Up @@ -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,
Expand Down Expand Up @@ -345,15 +345,17 @@ 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"))

cols = x.columns
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)

Expand Down Expand Up @@ -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 = {}
Expand Down