Skip to content

Commit

Permalink
Update many_independent_leaves test
Browse files Browse the repository at this point in the history
  • Loading branch information
gjoseph92 committed Jun 18, 2021
1 parent 768d660 commit 346ab17
Showing 1 changed file with 6 additions and 4 deletions.
10 changes: 6 additions & 4 deletions distributed/tests/test_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -100,14 +100,16 @@ async def test_recompute_released_results(c, s, a, b):
assert result == 1


@gen_cluster(client=True)
@gen_cluster(client=True, config={"distributed.scheduler.bandwidth": "1mb"})
async def test_decide_worker_with_many_independent_leaves(c, s, a, b):
# Make data large to penalize scheduling dependent tasks on other workers
ballast = b"\0" * int(s.bandwidth)
xs = await asyncio.gather(
c.scatter(list(range(0, 100, 2)), workers=a.address),
c.scatter(list(range(1, 100, 2)), workers=b.address),
c.scatter([bytes(i) + ballast for i in range(0, 100, 2)], workers=a.address),
c.scatter([bytes(i) + ballast for i in range(1, 100, 2)], workers=b.address),
)
xs = list(concat(zip(*xs)))
ys = [delayed(inc)(x) for x in xs]
ys = [delayed(lambda s: s[0])(x) for x in xs]

y2s = c.persist(ys)
await wait(y2s)
Expand Down

0 comments on commit 346ab17

Please sign in to comment.