-
-
Notifications
You must be signed in to change notification settings - Fork 719
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
Harden preamble of Worker.execute against race conditions #6878
Changes from all commits
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 | ||||
---|---|---|---|---|---|---|
|
@@ -108,7 +108,6 @@ | |||||
from distributed.worker_state_machine import ( | ||||||
NO_VALUE, | ||||||
AcquireReplicasEvent, | ||||||
AlreadyCancelledEvent, | ||||||
BaseWorker, | ||||||
CancelComputeEvent, | ||||||
ComputeTaskEvent, | ||||||
|
@@ -1847,9 +1846,7 @@ def _(**kwargs): | |||||
return _ | ||||||
|
||||||
@fail_hard | ||||||
def _handle_stimulus_from_task( | ||||||
self, task: asyncio.Task[StateMachineEvent | None] | ||||||
) -> None: | ||||||
def _handle_stimulus_from_task(self, task: asyncio.Task[StateMachineEvent]) -> None: | ||||||
"""Override BaseWorker method for added validation | ||||||
|
||||||
See also | ||||||
|
@@ -1968,15 +1965,22 @@ async def gather_dep( | |||||
total_nbytes: int, | ||||||
*, | ||||||
stimulus_id: str, | ||||||
) -> StateMachineEvent | None: | ||||||
) -> StateMachineEvent: | ||||||
"""Implements BaseWorker abstract method | ||||||
|
||||||
See also | ||||||
-------- | ||||||
distributed.worker_state_machine.BaseWorker.gather_dep | ||||||
""" | ||||||
if self.status not in WORKER_ANY_RUNNING: | ||||||
return None | ||||||
# This is only for the sake of coherence of the WorkerState; | ||||||
# it should never actually reach the scheduler. | ||||||
return GatherDepFailureEvent.from_exception( | ||||||
RuntimeError("Worker is shutting down"), | ||||||
worker=worker, | ||||||
total_nbytes=total_nbytes, | ||||||
stimulus_id=f"worker-closing-{time()}", | ||||||
) | ||||||
|
||||||
try: | ||||||
self.state.log.append( | ||||||
|
@@ -2044,7 +2048,7 @@ async def gather_dep( | |||||
stimulus_id=f"gather-dep-failure-{time()}", | ||||||
) | ||||||
|
||||||
async def retry_busy_worker_later(self, worker: str) -> StateMachineEvent | None: | ||||||
async def retry_busy_worker_later(self, worker: str) -> StateMachineEvent: | ||||||
"""Wait some time, then take a peer worker out of busy state. | ||||||
Implements BaseWorker abstract method. | ||||||
|
||||||
|
@@ -2137,24 +2141,21 @@ async def _maybe_deserialize_task( | |||||
return function, args, kwargs | ||||||
|
||||||
@fail_hard | ||||||
async def execute(self, key: str, *, stimulus_id: str) -> StateMachineEvent | None: | ||||||
async def execute(self, key: str, *, stimulus_id: str) -> StateMachineEvent: | ||||||
"""Execute a task. Implements BaseWorker abstract method. | ||||||
|
||||||
See also | ||||||
-------- | ||||||
distributed.worker_state_machine.BaseWorker.execute | ||||||
""" | ||||||
if self.status in {Status.closing, Status.closed, Status.closing_gracefully}: | ||||||
return None | ||||||
ts = self.state.tasks.get(key) | ||||||
if not ts: | ||||||
return None | ||||||
if ts.state == "cancelled": | ||||||
logger.debug( | ||||||
"Trying to execute task %s which is not in executing state anymore", | ||||||
ts, | ||||||
) | ||||||
return AlreadyCancelledEvent(key=ts.key, stimulus_id=stimulus_id) | ||||||
if self.status not in WORKER_ANY_RUNNING: | ||||||
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. align |
||||||
# This is just for internal coherence of the WorkerState; the reschedule | ||||||
# message should not ever reach the Scheduler. | ||||||
# It is still OK if it does though. | ||||||
return RescheduleEvent(key=key, stimulus_id=f"worker-closing-{time()}") | ||||||
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 new tests remain green if you revert the first line of this block: if self.status in {Status.closing, Status.closed, Status.closing_gracefully}:
return RescheduleEvent(key=key, stimulus_id=f"worker-closing-{time()}") In this case, the reschedule event actually reaches the scheduler. 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. I'm concerned that this causes the scheduler to reschedule a task twice. First, because the worker left, and second, because the reschedule event arrives. Edit: distributed/distributed/scheduler.py Lines 6740 to 6741 in 75ef3c9
|
||||||
|
||||||
# The key *must* be in the worker state thanks to the cancelled state | ||||||
ts = self.state.tasks[key] | ||||||
|
||||||
try: | ||||||
function, args, kwargs = await self._maybe_deserialize_task(ts) | ||||||
|
@@ -2169,7 +2170,7 @@ async def execute(self, key: str, *, stimulus_id: str) -> StateMachineEvent | No | |||||
try: | ||||||
if self.state.validate: | ||||||
assert not ts.waiting_for_data | ||||||
assert ts.state == "executing", ts.state | ||||||
assert ts.state in ("executing", "cancelled", "resumed"), ts | ||||||
assert ts.run_spec is not None | ||||||
|
||||||
args2, kwargs2 = self._prepare_args_for_execution(ts, args, kwargs) | ||||||
|
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.
The two new tests remain green if, instead of removing this block completely, you just replace the exit event with
They also remain green if you copy-paste the same block after the call to
_maybe_deserialize_task
.In both cases, the reschedule event reaches the scheduler and behaves as expected.