Skip to content

Commit dddf01e

Browse files
committed
TST/CLN: Catch more warnings
Catches a handful of warnings from the dask.dataframe test suite. Remaining warnings fall into a few classes: 1. ResoucreWarnings from cloudpickls (e.g. dask/dataframe/tests/test_arithmetics_reduction.py::test_frame_series_arithmetic_methods) Not sure what to do here. Probably a legitimate concern of dasks? 2. RuntimeWarnings from numpy (e.g. dask/dataframe/tests/test_arithmetics_reduction.py::test_reductions[False] I think dask should catch these, (followup PR) 3. ImportWarning from dependencies importing C code? e.g. dask/dataframe/tests/test_categorical.py::test_categorical_set_index[disk] I have a fix for partd (msgpack). Pandas has some too. Still investigating, may be a Cython issue 4. DeprecationWarning on regexes. e.g. dask/dataframe/tests/test_groupby.py::test_full_groupby_multilevel[grouper4] I think these are all pandas. xref pandas-dev/pandas#16481 (comment) 5. RuntimeWarning from pandas merge, e.g. dask/dataframe/tests/test_multi.py::test_merge_by_multiple_columns[disk-inner] Filing an issue on pandas
1 parent e7e2cfe commit dddf01e

10 files changed

+223
-166
lines changed

dask/dataframe/groupby.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -911,7 +911,7 @@ def apply(self, func, meta=no_default):
911911
" Before: .apply(func)\n"
912912
" After: .apply(func, meta={'x': 'f8', 'y': 'f8'}) for dataframe result\n"
913913
" or: .apply(func, meta=('x', 'f8')) for series result")
914-
warnings.warn(msg)
914+
warnings.warn(msg, stacklevel=2)
915915

916916
with raise_on_meta_error("groupby.apply({0})".format(funcname(func))):
917917
meta = self._meta_nonempty.apply(func)

dask/dataframe/io/tests/test_csv.py

Lines changed: 8 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -436,11 +436,12 @@ def test_warn_non_seekable_files():
436436
assert 'blocksize=None' in msg
437437

438438
with pytest.warns(None) as w:
439-
df = dd.read_csv('2014-01-*.csv', compression='gzip', blocksize=None)
439+
df = dd.read_csv('2014-01-*.csv', compression='gzip',
440+
blocksize=None)
440441
assert len(w) == 0
441442

442443
with pytest.raises(NotImplementedError):
443-
with pytest.warns(None):
444+
with pytest.warns(UserWarning): # needed for pytest
444445
df = dd.read_csv('2014-01-*.csv', compression='foo')
445446

446447

@@ -730,8 +731,8 @@ def test_read_csv_sep():
730731
charlie###300""")
731732

732733
with filetext(sep_text) as fn:
733-
ddf = dd.read_csv(fn, sep="###")
734-
df = pd.read_csv(fn, sep="###")
734+
ddf = dd.read_csv(fn, sep="###", engine="python")
735+
df = pd.read_csv(fn, sep="###", engine="python")
735736

736737
assert (df.columns == ddf.columns).all()
737738
assert len(df) == len(ddf)
@@ -862,13 +863,15 @@ def test_to_csv_multiple_files_cornercases():
862863

863864
@pytest.mark.xfail(reason="to_csv does not support compression")
864865
def test_to_csv_gzip():
866+
import warnings
867+
warnings.simplefilter("error", category=DeprecationWarning)
865868
df = pd.DataFrame({'x': ['a', 'b', 'c', 'd'],
866869
'y': [1, 2, 3, 4]}, index=[1., 2., 3., 4.])
867870

868871
for npartitions in [1, 2]:
869872
a = dd.from_pandas(df, npartitions)
870873
with tmpfile('csv') as fn:
871-
a.to_csv(fn, compression='gzip')
874+
a.to_csv(fn, compression='gzip', sep=",")
872875
result = pd.read_csv(fn, index_col=0, compression='gzip')
873876
tm.assert_frame_equal(result, df)
874877

dask/dataframe/io/tests/test_io.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -315,7 +315,7 @@ def test_from_pandas_with_datetime_index():
315315
"2015-08-25", "2015-08-24", "2015-08-21",
316316
"2015-08-20", "2015-08-19", "2015-08-18"],
317317
"Val": list(range(9))})
318-
df.Date = df.Date.astype('datetime64')
318+
df.Date = df.Date.astype('datetime64[ns]')
319319
ddf = dd.from_pandas(df, 2)
320320
assert_eq(df, ddf)
321321
ddf = dd.from_pandas(df, chunksize=2)

dask/dataframe/methods.py

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,7 @@
11
from __future__ import print_function, absolute_import, division
22

3+
import warnings
4+
35
import numpy as np
46
import pandas as pd
57
from pandas.api.types import is_categorical_dtype
@@ -253,8 +255,11 @@ def concat(dfs, axis=0, join='outer', uniform=False):
253255
# concatenates.
254256
dfs3 = [df if isinstance(df, pd.DataFrame) else
255257
df.to_frame().rename(columns={df.name: 0}) for df in dfs2]
256-
cat_mask = pd.concat([(df.dtypes == 'category').to_frame().T
257-
for df in dfs3], join=join).any()
258+
# pandas may raise a RuntimeWarning for comparing ints and strs
259+
with warnings.catch_warnings():
260+
warnings.simplefilter("ignore", RuntimeWarning)
261+
cat_mask = pd.concat([(df.dtypes == 'category').to_frame().T
262+
for df in dfs3], join=join).any()
258263

259264
if cat_mask.any():
260265
not_cat = cat_mask[~cat_mask].index
@@ -280,7 +285,10 @@ def concat(dfs, axis=0, join='outer', uniform=False):
280285
out[col] = union_categoricals(parts)
281286
out = out.reindex_axis(cat_mask.index, axis=1)
282287
else:
283-
out = pd.concat(dfs3, join=join)
288+
# pandas may raise a RuntimeWarning for comparing ints and strs
289+
with warnings.catch_warnings():
290+
warnings.simplefilter("ignore", RuntimeWarning)
291+
out = pd.concat(dfs3, join=join)
284292
else:
285293
if is_categorical_dtype(dfs2[0].dtype):
286294
if ind is None:

dask/dataframe/tests/test_arithmetics_reduction.py

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -635,9 +635,12 @@ def test_reductions(split_every):
635635
assert_eq(dds.min(split_every=split_every), pds.min())
636636
assert_eq(dds.max(split_every=split_every), pds.max())
637637
assert_eq(dds.count(split_every=split_every), pds.count())
638-
assert_eq(dds.std(split_every=split_every), pds.std())
639-
assert_eq(dds.var(split_every=split_every), pds.var())
640-
assert_eq(dds.sem(split_every=split_every), pds.sem())
638+
with pytest.warns(None):
639+
assert_eq(dds.std(split_every=split_every), pds.std())
640+
with pytest.warns(None):
641+
assert_eq(dds.var(split_every=split_every), pds.var())
642+
with pytest.warns(None):
643+
assert_eq(dds.sem(split_every=split_every), pds.sem())
641644
assert_eq(dds.std(ddof=0, split_every=split_every), pds.std(ddof=0))
642645
assert_eq(dds.var(ddof=0, split_every=split_every), pds.var(ddof=0))
643646
assert_eq(dds.sem(ddof=0, split_every=split_every), pds.sem(ddof=0))

dask/dataframe/tests/test_dataframe.py

Lines changed: 17 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -1381,9 +1381,10 @@ def test_eval():
13811381
with pytest.raises(NotImplementedError):
13821382
d.eval('z = x + y', inplace=True)
13831383

1384-
if p.eval('z = x + y', inplace=None) is None:
1385-
with pytest.raises(NotImplementedError):
1386-
d.eval('z = x + y', inplace=None)
1384+
with pytest.warns(None):
1385+
if p.eval('z = x + y', inplace=None) is None:
1386+
with pytest.raises(NotImplementedError):
1387+
d.eval('z = x + y', inplace=None)
13871388

13881389

13891390
@pytest.mark.parametrize('include, exclude', [
@@ -1702,7 +1703,7 @@ def test_apply():
17021703
ddf = dd.from_pandas(df, npartitions=2)
17031704

17041705
func = lambda row: row['x'] + row['y']
1705-
assert_eq(ddf.x.apply(lambda x: x + 1),
1706+
assert_eq(ddf.x.apply(lambda x: x + 1, meta=("x", int)),
17061707
df.x.apply(lambda x: x + 1))
17071708

17081709
# specify meta
@@ -1712,16 +1713,17 @@ def test_apply():
17121713
df.apply(lambda xy: xy[0] + xy[1], axis='columns'))
17131714

17141715
# inference
1715-
assert_eq(ddf.apply(lambda xy: xy[0] + xy[1], axis=1),
1716+
assert_eq(ddf.apply(lambda xy: xy[0] + xy[1], axis=1, meta=(None, int)),
17161717
df.apply(lambda xy: xy[0] + xy[1], axis=1))
1717-
assert_eq(ddf.apply(lambda xy: xy, axis=1),
1718+
assert_eq(ddf.apply(lambda xy: xy, axis=1, meta={'x': int, 'y': int}),
17181719
df.apply(lambda xy: xy, axis=1))
17191720

17201721
# specify meta
17211722
func = lambda x: pd.Series([x, x])
17221723
assert_eq(ddf.x.apply(func, meta=[(0, int), (1, int)]), df.x.apply(func))
17231724
# inference
1724-
assert_eq(ddf.x.apply(func), df.x.apply(func))
1725+
with pytest.warns(UserWarning):
1726+
assert_eq(ddf.x.apply(func), df.x.apply(func))
17251727

17261728
# axis=0
17271729
with pytest.raises(NotImplementedError):
@@ -1886,13 +1888,15 @@ def return_df(x):
18861888
return pd.Series([x.sum(), x.mean()], index=['sum', 'mean'])
18871889

18881890
# DataFrame to completely different DataFrame
1889-
result = ddf.apply(return_df, axis=1)
1891+
with pytest.warns(UserWarning):
1892+
result = ddf.apply(return_df, axis=1)
18901893
assert isinstance(result, dd.DataFrame)
18911894
tm.assert_index_equal(result.columns, pd.Index(['sum', 'mean']))
18921895
assert_eq(result, df.apply(return_df, axis=1))
18931896

18941897
# DataFrame to Series
1895-
result = ddf.apply(lambda x: 1, axis=1)
1898+
with pytest.warns(UserWarning):
1899+
result = ddf.apply(lambda x: 1, axis=1)
18961900
assert isinstance(result, dd.Series)
18971901
assert result.name is None
18981902
assert_eq(result, df.apply(lambda x: 1, axis=1))
@@ -1901,13 +1905,15 @@ def return_df2(x):
19011905
return pd.Series([x * 2, x * 3], index=['x2', 'x3'])
19021906

19031907
# Series to completely different DataFrame
1904-
result = ddf.x.apply(return_df2)
1908+
with pytest.warns(UserWarning):
1909+
result = ddf.x.apply(return_df2)
19051910
assert isinstance(result, dd.DataFrame)
19061911
tm.assert_index_equal(result.columns, pd.Index(['x2', 'x3']))
19071912
assert_eq(result, df.x.apply(return_df2))
19081913

19091914
# Series to Series
1910-
result = ddf.x.apply(lambda x: 1)
1915+
with pytest.warns(UserWarning):
1916+
result = ddf.x.apply(lambda x: 1)
19111917
assert isinstance(result, dd.Series)
19121918
assert result.name == 'x'
19131919
assert_eq(result, df.x.apply(lambda x: 1))

0 commit comments

Comments
 (0)