Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions distributed/shuffle/_shuffle.py
Original file line number Diff line number Diff line change
Expand Up @@ -104,9 +104,15 @@ def rearrange_by_column_p2p(
column: str,
npartitions: int | None = None,
) -> DataFrame:
import pandas as pd

from dask.dataframe.core import new_dd_object

meta = df._meta
if not pd.api.types.is_integer_dtype(meta[column].dtype):
raise TypeError(
Copy link
Member Author

Choose a reason for hiding this comment

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

This used to be tested before dask/dask#10493 has been merged.

f"Expected meta {column=} to be an integer column, is {meta[column].dtype}."
)
check_dtype_support(meta)
npartitions = npartitions or df.npartitions
token = tokenize(df, column, npartitions)
Expand Down
42 changes: 42 additions & 0 deletions distributed/shuffle/tests/test_shuffle.py
Original file line number Diff line number Diff line change
Expand Up @@ -2165,3 +2165,45 @@ async def test_set_index_p2p(c, s, *workers):
await c.close()
await asyncio.gather(*[check_worker_cleanup(w) for w in workers])
await check_scheduler_cleanup(s)


def test_shuffle_p2p_with_existing_index(client):
df = pd.DataFrame({"a": np.random.randint(0, 3, 20)}, index=np.random.random(20))
ddf = dd.from_pandas(
df,
npartitions=4,
)
ddf = ddf.shuffle("a", shuffle="p2p")
result = client.compute(ddf, sync=True)
dd.assert_eq(result, df)


def test_set_index_p2p_with_existing_index(client):
df = pd.DataFrame({"a": np.random.randint(0, 3, 20)}, index=np.random.random(20))
ddf = dd.from_pandas(
df,
npartitions=4,
)
ddf = ddf.set_index("a", shuffle="p2p")
result = client.compute(ddf, sync=True)
dd.assert_eq(result, df.set_index("a"))


def test_sort_values_p2p_with_existing_divisions(client):
"Regression test for #8165"
df = pd.DataFrame(
{"a": np.random.randint(0, 3, 20), "b": np.random.randint(0, 3, 20)}
)
ddf = dd.from_pandas(
df,
npartitions=4,
)
with dask.config.set({"dataframe.shuffle.method": "p2p"}):
ddf = ddf.set_index("a").sort_values("b")
result = client.compute(ddf, sync=True)
dd.assert_eq(
result,
df.set_index("a").sort_values("b"),
check_index=False,
sort_results=False,
)