-
-
Notifications
You must be signed in to change notification settings - Fork 720
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Scheduler.reschedule() works only by accident #6339
Changes from all commits
6999e42
ddf06f1
c857e92
2fe8146
9410072
2e06275
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,123 @@ | ||
"""Tests for tasks raising the Reschedule exception and Scheduler.reschedule(). | ||
|
||
Note that this functionality is also used by work stealing; | ||
see test_steal.py for additional tests. | ||
""" | ||
from __future__ import annotations | ||
|
||
import asyncio | ||
from time import sleep | ||
|
||
import pytest | ||
|
||
from distributed import Event, Reschedule, get_worker, secede, wait | ||
from distributed.utils_test import captured_logger, gen_cluster, slowinc | ||
from distributed.worker_state_machine import ( | ||
ComputeTaskEvent, | ||
FreeKeysEvent, | ||
RescheduleEvent, | ||
SecedeEvent, | ||
) | ||
|
||
|
||
@gen_cluster( | ||
client=True, | ||
nthreads=[("", 1)] * 2, | ||
config={"distributed.scheduler.work-stealing": False}, | ||
) | ||
async def test_scheduler_reschedule(c, s, a, b): | ||
xs = c.map(slowinc, range(100), key="x", delay=0.1) | ||
while not a.state.tasks or not b.state.tasks: | ||
await asyncio.sleep(0.01) | ||
assert len(a.state.tasks) == len(b.state.tasks) == 50 | ||
|
||
ys = c.map(slowinc, range(100), key="y", delay=0.1, workers=[a.address]) | ||
while len(a.state.tasks) != 150: | ||
await asyncio.sleep(0.01) | ||
|
||
# Reschedule the 50 xs that are processing on a | ||
for x in xs: | ||
if s.tasks[x.key].processing_on is s.workers[a.address]: | ||
s.reschedule(x.key, stimulus_id="test") | ||
|
||
# Wait for at least some of the 50 xs that had been scheduled on a to move to b. | ||
# This happens because you have 100 ys processing on a and 50 xs processing on b, | ||
# so the scheduler will prefer b for the rescheduled tasks to obtain more equal | ||
# balancing. | ||
while len(a.state.tasks) == 150 or len(b.state.tasks) <= 50: | ||
await asyncio.sleep(0.01) | ||
|
||
|
||
@gen_cluster() | ||
async def test_scheduler_reschedule_warns(s, a, b): | ||
with captured_logger("distributed.scheduler") as sched: | ||
s.reschedule(key="__this-key-does-not-exist__", stimulus_id="test") | ||
|
||
assert "not found on the scheduler" in sched.getvalue() | ||
assert "Aborting reschedule" in sched.getvalue() | ||
|
||
|
||
@pytest.mark.parametrize("long_running", [False, True]) | ||
@gen_cluster( | ||
client=True, | ||
nthreads=[("", 1)] * 2, | ||
config={"distributed.scheduler.work-stealing": False}, | ||
) | ||
async def test_raise_reschedule(c, s, a, b, long_running): | ||
"""A task raises Reschedule()""" | ||
a_address = a.address | ||
|
||
def f(x): | ||
if long_running: | ||
secede() | ||
sleep(0.1) | ||
if get_worker().address == a_address: | ||
raise Reschedule() | ||
|
||
futures = c.map(f, range(4), key=["x1", "x2", "x3", "x4"]) | ||
futures2 = c.map(slowinc, range(10), delay=0.1, key="clog", workers=[a.address]) | ||
await wait(futures) | ||
assert any(isinstance(ev, RescheduleEvent) for ev in a.state.stimulus_log) | ||
assert all(f.key in b.data for f in futures) | ||
|
||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the two tests below are new |
||
@pytest.mark.parametrize("long_running", [False, True]) | ||
@gen_cluster(client=True, nthreads=[("", 1)]) | ||
async def test_cancelled_reschedule(c, s, a, long_running): | ||
"""A task raises Reschedule(), but the future was released by the client""" | ||
ev1 = Event() | ||
ev2 = Event() | ||
|
||
def f(ev1, ev2): | ||
if long_running: | ||
secede() | ||
ev1.set() | ||
ev2.wait() | ||
raise Reschedule() | ||
|
||
x = c.submit(f, ev1, ev2, key="x") | ||
await ev1.wait() | ||
x.release() | ||
while "x" in s.tasks: | ||
await asyncio.sleep(0.01) | ||
|
||
await ev2.set() | ||
while "x" in a.state.tasks: | ||
await asyncio.sleep(0.01) | ||
|
||
|
||
@pytest.mark.parametrize("long_running", [False, True]) | ||
def test_cancelled_reschedule_worker_state(ws, long_running): | ||
"""Same as test_cancelled_reschedule""" | ||
|
||
ws.handle_stimulus(ComputeTaskEvent.dummy(key="x", stimulus_id="s1")) | ||
if long_running: | ||
ws.handle_stimulus(SecedeEvent(key="x", compute_duration=1.0, stimulus_id="s2")) | ||
|
||
instructions = ws.handle_stimulus( | ||
FreeKeysEvent(keys=["x"], stimulus_id="s3"), | ||
RescheduleEvent(key="x", stimulus_id="s4"), | ||
) | ||
# There's no RescheduleMsg and the task has been forgotten | ||
assert not instructions | ||
assert not ws.tasks |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1132,9 +1132,12 @@ async def test_balance_many_workers(c, s, *workers): | |
|
||
|
||
@nodebug | ||
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 30) | ||
@gen_cluster( | ||
client=True, | ||
nthreads=[("127.0.0.1", 1)] * 30, | ||
config={"distributed.scheduler.work-stealing": False}, | ||
) | ||
async def test_balance_many_workers_2(c, s, *workers): | ||
await s.extensions["stealing"].stop() | ||
futures = c.map(slowinc, range(90), delay=0.2) | ||
await wait(futures) | ||
assert {len(w.has_what) for w in s.workers.values()} == {3} | ||
|
@@ -1513,35 +1516,6 @@ async def test_log_tasks_during_restart(c, s, a, b): | |
assert "exit" in str(s.events) | ||
|
||
|
||
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2) | ||
async def test_reschedule(c, s, a, b): | ||
await c.submit(slowinc, -1, delay=0.1) # learn cost | ||
x = c.map(slowinc, range(4), delay=0.1) | ||
|
||
# add much more work onto worker a | ||
futures = c.map(slowinc, range(10, 20), delay=0.1, workers=a.address) | ||
|
||
while len(s.tasks) < len(x) + len(futures): | ||
await asyncio.sleep(0.001) | ||
|
||
for future in x: | ||
s.reschedule(key=future.key) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This test remained green if you removed these lines |
||
|
||
# Worker b gets more of the original tasks | ||
await wait(x) | ||
assert sum(future.key in b.data for future in x) >= 3 | ||
assert sum(future.key in a.data for future in x) <= 1 | ||
|
||
|
||
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2) | ||
async def test_reschedule_warns(c, s, a, b): | ||
with captured_logger(logging.getLogger("distributed.scheduler")) as sched: | ||
s.reschedule(key="__this-key-does-not-exist__") | ||
|
||
assert "not found on the scheduler" in sched.getvalue() | ||
assert "Aborting reschedule" in sched.getvalue() | ||
|
||
|
||
@gen_cluster(client=True) | ||
async def test_get_task_status(c, s, a, b): | ||
future = c.submit(inc, 1) | ||
|
@@ -3342,12 +3316,11 @@ async def test_worker_heartbeat_after_cancel(c, s, *workers): | |
|
||
@gen_cluster(client=True, nthreads=[("", 1)] * 2) | ||
async def test_set_restrictions(c, s, a, b): | ||
|
||
f = c.submit(inc, 1, workers=[b.address]) | ||
f = c.submit(inc, 1, key="f", workers=[b.address]) | ||
await f | ||
s.set_restrictions(worker={f.key: a.address}) | ||
assert s.tasks[f.key].worker_restrictions == {a.address} | ||
s.reschedule(f) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This was silently doing nothing (future |
||
await b.close() | ||
await f | ||
|
||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Out-of-scope comment: Is there a better name that highlights that
reschedule
does not actually reschedule, i.e., it does not schedule the task somewhere else, it merely cancels the previous scheduling decision? For example,deschedule
might be better.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Well no. A transition to released will automatically kick the task back to waiting.
The functional description of the method is accurate. The "released" bit is an implementation detail.
I'm adding a comment to explain.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense from that perspective, I was thinking that the function doesn't include the rescheduling bit, but in the end you're right, the releasing/descheduling automatically achieves the aim of rescheduling.