Skip to content
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

Migrate ensure_computing transitions to new WorkerState event mechanism - part 1 #6003

Merged
merged 19 commits into from
Apr 4, 2022

Conversation

crusaderky
Copy link
Collaborator

@crusaderky crusaderky commented Mar 25, 2022

Partially closes #5895
See comment below for scope of this PR

self._async_instruction_callback,
self.execute(inst.key, stimulus_id=inst.stimulus_id),
stimulus_id=inst.stimulus_id,
)
Copy link
Collaborator Author

@crusaderky crusaderky Mar 25, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This differs from the design document, which instead was creating a fire-and-forget asyncio.Task.
As far as I understand, the difference between the two is purely cosmetic.

Added value would be given from spawning a task, track it e.g. in a set Worker.running_asyncio_tasks, and then cancel it in Worker.close(). Even if desirable, however, I think this is best left to a future PR.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added value would be given from spawning a task, track it e.g. in a set Worker.running_asyncio_tasks, and then cancel it in Worker.close(). Even if desirable, however, I think this is best left to a future PR.

That was my intention, I just didn't specify everything in my pseudo code. I was actually hoping we'd implement this already as part of #5922. I'm OK with postponing this to a follow up but I would like to get this done rather sooner than later.

I would like to avoid using add_callback if at all possible since tracking the tasks would actually allow us to, e.g. deal with the exception

msg = error_message(exc)
recommendations = {ts: tuple(msg.values())}

return recommendations, []
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This differs from #5895:

  • The Worker.execute method is modified such that it no longer performs any transition but instead returns appropriate StateMachineEvents that trigger the necessary handlers. For instance
    • TaskFinished
    • Rescheduled
    • TaskErred

I could not find any benefit in implementing those events vs. just returning recommendations?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The benefit is that we want to log the events and keep the recommendations as an internal detail of the to-be-defined WorkerState class. Not using recommendations here is one of the more important points of the design proposals

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please have a look now

@crusaderky crusaderky requested a review from fjetter March 25, 2022 14:35
self._async_instruction_callback,
self.execute(inst.key, stimulus_id=inst.stimulus_id),
stimulus_id=inst.stimulus_id,
)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added value would be given from spawning a task, track it e.g. in a set Worker.running_asyncio_tasks, and then cancel it in Worker.close(). Even if desirable, however, I think this is best left to a future PR.

That was my intention, I just didn't specify everything in my pseudo code. I was actually hoping we'd implement this already as part of #5922. I'm OK with postponing this to a follow up but I would like to get this done rather sooner than later.

I would like to avoid using add_callback if at all possible since tracking the tasks would actually allow us to, e.g. deal with the exception

distributed/worker.py Outdated Show resolved Hide resolved
msg = error_message(exc)
recommendations = {ts: tuple(msg.values())}

return recommendations, []
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The benefit is that we want to log the events and keep the recommendations as an internal detail of the to-be-defined WorkerState class. Not using recommendations here is one of the more important points of the design proposals

@github-actions
Copy link
Contributor

github-actions bot commented Mar 28, 2022

Unit Test Results

       17 files   -        1         17 suites   - 1   8h 42m 40s ⏱️ - 28m 20s
  2 705 tests +       8    2 621 ✔️ +       7       82 💤 +  1  2 ±0 
22 700 runs   - 1 421  21 566 ✔️  - 1 339  1 132 💤  - 82  2 ±0 

For more details on these failures, see this check.

Results for commit f83df7a. ± Comparison against base commit ccb0362.

♻️ This comment has been updated with latest results.

# yet.
assert not ts.dependents
self.transition(ts, "released", stimulus_id=stimulus_id)
self.handle_stimulus(CancelComputeEvent(key=key, stimulus_id=stimulus_id))
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the future we should consider sending event objects directly from the scheduler

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I could see this very nicely being integrated in our RPC framework 👍

Copy link
Member

@fjetter fjetter left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think there is some confusion abut the CancelComputeEvent but otherwise this looks already good

# yet.
assert not ts.dependents
self.transition(ts, "released", stimulus_id=stimulus_id)
self.handle_stimulus(CancelComputeEvent(key=key, stimulus_id=stimulus_id))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I could see this very nicely being integrated in our RPC framework 👍

if not ts:
return None
if ts.state == "cancelled":
return CancelComputeEvent(key=ts.key, stimulus_id=stimulus_id)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would suggest a name like AlreadyCancelledEvent to distinguish this from Client.cancel events passed through the scheduler. We don't have this, yet, but there is some potential for confusion

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I split them now

recommendations: Recs = {}
self.active_keys.discard(key)

self.threads[key] = result["thread"]
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we want to move this to the result handlers as well? This will be a duplicated line but we'd cleanly separate result handling from the exceution and would only mutate state in a single place

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understand that neither the Worker ABC nor the WorkerStateMachine will ever have a concept of threads. That's why I left it here.

if result["op"] == "task-finished":
ts.nbytes = result["nbytes"]
ts.type = result["type"]
recommendations[ts] = ("memory", value)
if self.digests is not None:
self.digests["task-duration"].add(result["stop"] - result["start"])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here. Do we want to move this to the result/event handler?

Copy link
Collaborator Author

@crusaderky crusaderky Mar 30, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

self.digests is populated with a wealth of information that should remain alien to the state machine:

  • latency
  • transfer-bandwidth
  • get-data-send-duration
  • disk-load-duration
  • profile-duration

so I think it should remain in Worker?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can also nuke the digests if they're slowing us down. They're not commonly used today.

raise TypeError(ev) # pragma: nocover

# TODO Set return type annotation of all handle_event implementations
# to tuple[Recs, Instructions] (requires Python >=3.9)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this a functools limitation or why do we need py3.9? Do we even need individual annotations if they are all the same?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this a functools limitation or why do we need py3.9?

@functools.singledispatchmethod exec()'s the delayed annotation - which however require Python 3.9+.
I worked around it better now.

Do we even need individual annotations if they are all the same?

mypy is not smart enough to reserve a special treatment for singledispatch functions, so yes

# to tuple[Recs, Instructions] (requires Python >=3.9)
@handle_event.register
def _(self, ev: CancelComputeEvent): # -> tuple[Recs, Instructions]:
ts = self.tasks.get(ev.key)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The tasks must always exist. There is no way this could've been dropped earlier. If so, that would be a severe bug

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I had conflated the implementation of handle_cancel_compute coming from the scheduler and the code that dealt with a task being already cancelled by the time execute starts. They are now separate.

self.log.append((ev.key, "cancel-compute", ev.stimulus_id, time()))
# All possible dependents of ts should not be in state Processing on
# scheduler side and therefore should not be assigned to a worker, yet.
assert not ts.dependents
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I suggest to not introduce any new assertions. For this specific handler there is no need to assert this condition, is there?

I'm not even sure if this assert is correct. I've seen such race conditions happening. For instance, there is a network partition such that this worker briefly disconnects. Another worker gets this task assigned and finishes earlier. In the meantime, this worker reconnects but the task is still executing/not yet executing but got a depentent assigned in the meantime.

See also the warning on scheduler side Unexpected worker completed task [...]

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I had conflated the implementation of handle_cancel_compute coming from the scheduler and the code that dealt with a task being already cancelled by the time execute starts. They are now separate.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It should now be functionally identical to main

# TODO Set return type annotation of all handle_event implementations
# to tuple[Recs, Instructions] (requires Python >=3.9)
@handle_event.register
def _(self, ev: CancelComputeEvent): # -> tuple[Recs, Instructions]:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm a bit confused about the body of this handler. I don't think we should mix up the "CancelTask" event with a execute response that says "Didn't Execute. Task already cancelled"

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I split them now

ts.startstops.append({"action": "compute", "start": ev.start, "stop": ev.stop})
ts.nbytes = ev.nbytes
ts.type = ev.type
return {ts: ("memory", ev.value)}, []
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We are loosing stimulus_ids here aren't we?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We aren't; see handle_stimulus:

            recs, instructions = self.handle_event(stim)
            self.transitions(recs, stimulus_id=stim.stimulus_id)

distributed/worker.py Show resolved Hide resolved
This reverts commit 9d28bde.
@crusaderky
Copy link
Collaborator Author

crusaderky commented Mar 30, 2022

@fjetter I did the work on _ensure_computing (see 9d28bde) but there are still some pretty conceptual failures caused by resource counting, so I'm moderately inclined to merge this PR as it is. I reverted it for now.
All tests are green.

In scope for this PR

  • Remove all self.loop.add_callback(self.execute, ...)

Out of scope for the PR but in scope for the issue (to be implemented immediately after this PR)

  • Refactor Worker.ensure_computing() -> None into Worker._ensure_computing() -> RecsInstr
  • Implement Worker.stimulus_history
  • Track tasks spawned by Worker._handle_instructions and cancel them in Worker.close

Out of scope for the issue

  • Send Event Python objects from the Scheduler
  • Other events (replicate etc.)

@crusaderky crusaderky marked this pull request as ready for review March 30, 2022 13:59
@crusaderky crusaderky changed the title Migrate ensure_executing transitions to new WorkerState event mechanism Migrate ensure_executing transitions to new WorkerState event mechanism - part 1 Mar 30, 2022
crusaderky added a commit to crusaderky/distributed that referenced this pull request Mar 30, 2022
@crusaderky crusaderky self-assigned this Mar 30, 2022
Copy link
Member

@fjetter fjetter left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@crusaderky crusaderky merged commit c20a099 into dask:main Apr 4, 2022
@crusaderky crusaderky deleted the worker_state_machine branch April 4, 2022 15:17
@crusaderky crusaderky changed the title Migrate ensure_executing transitions to new WorkerState event mechanism - part 1 Migrate ensure_computing transitions to new WorkerState event mechanism - part 1 Apr 8, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Migrate ensure_computing transitions to new WorkerState event mechanism
3 participants