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

Restructure Worker.close #8062

Closed
hendrikmakait opened this issue Aug 2, 2023 · 3 comments · Fixed by #8066 or #8076
Closed

Restructure Worker.close #8062

hendrikmakait opened this issue Aug 2, 2023 · 3 comments · Fixed by #8066 or #8076

Comments

@hendrikmakait
Copy link
Member

hendrikmakait commented Aug 2, 2023

At the moment, the order of operations in Worker.close is somewhat random and does not appear intentionally structured.

It reads roughly as follows:

  1. Update self.status to Status.closing
    1. Inform scheduler of status change
    2. Handle PauseEvent to prevent further tasks from being executed/gathered
  2. Stop self.periodic_callbacks
  3. self.stop()
    1. Purge workspace
    2. Close monitor
    3. Stop sync listeners
    4. Abort handshaking comms
    5. Stop async listeners in the background
  4. Run BaseWorker.close(timeout=...)
    1. Cancel self.async_instructions
    2. Wait for them to finish (with timeout)
  5. Teardown preloads
  6. Close extensions
  7. Maybe tell nanny to close_gracefully
  8. Teardown plugins
  9. Close worker-initialized clients
  10. Close scheduler RPC
  11. Stop services
  12. Wait a bit in case we’re using UCX
  13. Close batched stream by first sending close-stream, then closing with a timeout
  14. Shutdown executors (maybe wait with timeout)
  15. Close RPC
  16. Update status to Status.closed
  17. ServerNode.close()
    1. stop periodic_callbacks (again)
    2. stop monitor (again)
    3. stop listeners (again)
    4. Abort handshaking comms (again)
    5. Stop async listeners (again but blocking this time)
    6. Stop background_tasks
    7. close RPC (again)
    8. Close comms
    9. Remove local_directory from sys.path
  18. Unwind exit_stack

There are multiple issues with this:

  • The workspace gets purged before plugins get torn down, which may remove data of an in-progress P2P shuffle and cause exceptions.
  • It's generally hard to understand what state a worker is in at any given point in time
  • Just to add to the confusion, some stuff is stopped multiple times.

As an alternative, I suggest the following rough order of operations:

  1. Inform the cluster that the worker is closing
    • Inform scheduler
    • Inform nanny
    • Set self.status = Status.closing
  2. Tear down any running "add-ins" while resources are still working
    • plugins
    • extensions
    • services
    • preloads
  3. Prevent new communications from being established
  4. Close existing comms
  5. Close local resources and clean up internal state

In particular, add-ins such as plugins should be torn down while the worker is still functional to allow them to take arbitrary actions such as informing the scheduler or initiating communications with other workers.

Note:
Timeouts are also a problem but should be tackled in a dedicated PR (see #7318, #7320).

@hendrikmakait
Copy link
Member Author

hendrikmakait commented Aug 2, 2023

In particular, add-ins such as plugins should be torn down while the worker is still functional to allow them to take arbitrary actions such as informing the scheduler or initiating communications with other workers.

An alternative to this would be to remove the worker from the scheduler ASAP, mimicking the case where a worker dies. Any necessary cluster-wide coordination would then be left to SchedulerPlugins, and WorkerPlugins would be scoped to local teardown.

@hendrikmakait
Copy link
Member Author

Fun fact: By stopping periodic_callbacks in step 2, we also stop heartbeats and keep-alives. If closing the worker takes too long, its TTL will expire and the scheduler will trigger remove_worker regardless of how far along the worker is in its shutdown.

@fjetter
Copy link
Member

fjetter commented Aug 3, 2023

Inform the cluster that the worker is closing

+1

Inform scheduler

this is done by setting the status. I don't think we should do anything else at this stage.


overall the proposed ordering makes sense to me. I think the PCs have to be stopped between 2 and 3. PCs do need comms and we should stop them before closing them.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
3 participants