Skip to content

Commit

Permalink
Avoid checking against types in is_dataframe_like (dask#4418)
Browse files Browse the repository at this point in the history
  • Loading branch information
mrocklin committed Jan 28, 2019
1 parent 760932f commit 16dc34c
Show file tree
Hide file tree
Showing 3 changed files with 12 additions and 3 deletions.
6 changes: 6 additions & 0 deletions dask/dataframe/tests/test_dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -553,6 +553,12 @@ def test_map_partitions():
assert result.dtype == np.float64 and result.compute() == 4.0


def test_map_partitions_type():
result = d.map_partitions(type).compute(scheduler='single-threaded')
assert isinstance(result, pd.Series)
assert all(x == pd.DataFrame for x in result)


def test_map_partitions_names():
func = lambda x: x
assert (sorted(dd.map_partitions(func, d, meta=d).dask) ==
Expand Down
3 changes: 3 additions & 0 deletions dask/dataframe/tests/test_utils_dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -337,14 +337,17 @@ def test_is_dataframe_like():
assert is_dataframe_like(df)
assert not is_dataframe_like(df.x)
assert not is_dataframe_like(df.index)
assert not is_dataframe_like(pd.DataFrame)

assert not is_series_like(df)
assert is_series_like(df.x)
assert not is_series_like(df.index)
assert not is_series_like(pd.Series)

assert not is_index_like(df)
assert not is_index_like(df.x)
assert is_index_like(df.index)
assert not is_index_like(pd.Index)

ddf = dd.from_pandas(df, npartitions=1)
assert is_dataframe_like(ddf)
6 changes: 3 additions & 3 deletions dask/dataframe/utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -497,18 +497,18 @@ def _nonempty_series(s, idx=None):

def is_dataframe_like(df):
""" Looks like a Pandas DataFrame """
return set(dir(df)) > {'dtypes', 'columns', 'groupby', 'head'}
return set(dir(df)) > {'dtypes', 'columns', 'groupby', 'head'} and not isinstance(df, type)


def is_series_like(s):
""" Looks like a Pandas Series """
return set(dir(s)) > {'name', 'dtype', 'groupby', 'head'}
return set(dir(s)) > {'name', 'dtype', 'groupby', 'head'} and not isinstance(s, type)


def is_index_like(s):
""" Looks like a Pandas Index """
attrs = set(dir(s))
return attrs > {'name', 'dtype'} and 'head' not in attrs
return attrs > {'name', 'dtype'} and 'head' not in attrs and not isinstance(s, type)


def check_meta(x, meta, funcname=None, numeric_equal=True):
Expand Down

0 comments on commit 16dc34c

Please sign in to comment.