diff --git a/distributed/shuffle/tests/test_shuffle.py b/distributed/shuffle/tests/test_shuffle.py index b89913b77c..3e211e1683 100644 --- a/distributed/shuffle/tests/test_shuffle.py +++ b/distributed/shuffle/tests/test_shuffle.py @@ -30,6 +30,7 @@ pd = pytest.importorskip("pandas") import dask.dataframe as dd +from dask.dataframe._compat import PANDAS_GE_300 from dask.typing import Key from distributed import ( @@ -2435,6 +2436,7 @@ async def test_handle_null_partitions(c, s, a, b): await assert_scheduler_cleanup(s) +@pytest.mark.xfail(PANDAS_GE_300, reason="dask/distributed#9184", strict=True) @gen_cluster(client=True) async def test_handle_null_partitions_2(c, s, a, b): def make_partition(i): @@ -2447,7 +2449,7 @@ def make_partition(i): ddf = dd.from_map(make_partition, range(5), meta={"a": float, "b": float}) with dask.config.set({"dataframe.shuffle.method": "p2p"}): out = ddf.shuffle(on="a", ignore_index=True, force=True) - result, expected = c.compute([ddf, out]) + expected, result = c.compute([ddf, out]) del out result = await result expected = await expected