Skip to content

Commit

Permalink
Ensure shuffle split operations are blacklisted from work stealing (#…
Browse files Browse the repository at this point in the history
…4964)

If shuffle split tasks are not blacklisted from work stealing, this can have
catastrophic effects on performance.
See also #4962
  • Loading branch information
fjetter authored Jun 29, 2021
1 parent 06af2ba commit d419e41
Show file tree
Hide file tree
Showing 2 changed files with 36 additions and 1 deletion.
2 changes: 1 addition & 1 deletion distributed/stealing.py
Original file line number Diff line number Diff line change
Expand Up @@ -453,4 +453,4 @@ def _can_steal(thief, ts, victim):
return True


fast_tasks = {"shuffle-split"}
fast_tasks = {"split-shuffle"}
35 changes: 35 additions & 0 deletions distributed/tests/test_steal.py
Original file line number Diff line number Diff line change
Expand Up @@ -827,3 +827,38 @@ async def test_balance_with_longer_task(c, s, a, b):
) # a task after y, suggesting a, but open to b
await z
assert z.key in b.data


@gen_cluster(client=True)
async def test_blacklist_shuffle_split(c, s, a, b):

pd = pytest.importorskip("pandas")
dd = pytest.importorskip("dask.dataframe")
npart = 10
df = dd.from_pandas(pd.DataFrame({"A": range(100), "B": 1}), npartitions=npart)
graph = df.shuffle(
"A",
shuffle="tasks",
# If we don't have enough partitions, we'll fall back to a simple shuffle
max_branch=npart - 1,
).sum()
res = c.compute(graph)

while not s.tasks:
await asyncio.sleep(0.005)
prefixes = set(s.task_prefixes.keys())
from distributed.stealing import fast_tasks

blacklisted = fast_tasks & prefixes
assert blacklisted
assert any(["split" in prefix for prefix in blacklisted])

stealable = s.extensions["stealing"].stealable
while not res.done():
for tasks_per_level in stealable.values():
for tasks in tasks_per_level:
for ts in tasks:
assert ts.prefix.name not in fast_tasks
assert "split" not in ts.prefix.name
await asyncio.sleep(0.001)
await res

0 comments on commit d419e41

Please sign in to comment.