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

Alternative worker_client usage model #7311

Open
jakirkham opened this issue Nov 15, 2022 · 3 comments
Open

Alternative worker_client usage model #7311

jakirkham opened this issue Nov 15, 2022 · 3 comments

Comments

@jakirkham
Copy link
Member

Currently if users would like to do some custom task submission as part of their task graph, Distributed provides worker_client. This does some secede/rejoin logic to allow the Worker to leave the pool and the rejoin later. However this comes with some caveats:

  1. A patched copy of concurrent.futures
  2. The former requires repatching to get recent concurrent.futures updates
  3. Lack of visibility to users running code that may rely on this feature (if they want to understand their workflow's usage)
  4. Inability to use these workflows with the local scheduler or alternative Executors easily

An alternative approach might be to surface the needed to perform dynamic submission at a higher level (like as part of the graph itself). For example one might decorate functions that would like access to the Client and provide a client argument, which the decorator would fill in (as part of task submission). This would make it more visible to users of workflows depending on this functionality. Also the Client and Scheduler could take this into account when orchestrating task submission. This would also avoid the need to think about how workers interact with this feature (since the Client and Scheduler could take care of that in advance).

There may be other approaches that are better than this option. Raising mostly to get the discussion on better alternatives that might simplify maintenance/usage

@fjetter
Copy link
Member

fjetter commented Nov 15, 2022

I dislike the way worker_client behaves as well. There are still many problems around secede/rejoin. IIUC your proposal would basically use the same functionality but offer a different API?

Would something like #5671 serve your use case as well?

@jakirkham
Copy link
Member Author

Would like disambiguate two things in this proposal:

  1. Surfacing the need for a specific function in the graph to submit tasks/work with futures/etc.
  2. A particular proposal using a decorator as an example

Think 1 is the important thing as most of the pain with worker_client is the fact that functions may be doing this internally without any awareness by the Client or Scheduler ahead of time. That results in various workarounds (like secede/rejoin) and can cause unexpected behavior (resource overuse). If the Client or Scheduler know in advance this kind of work can happen, they may even be able to do some or all of this themselves (without needing to do something special in the worker) and leverage existing functionality to moderate task submission.

With 2, just find it is easier to think about a concrete example of the proposal. Am not attached to this particular proposal. There may be others that are better.


Would something like #5671 serve your use case as well?

Not quite as it is still is requesting a client within the task function as opposed to somehow flagging this function in the graph. Though it may be complimentary as handling Futures returned seems important as well.

@fjetter
Copy link
Member

fjetter commented Nov 16, 2022

I actually thought earlier about removing secede and rejoin entirely (at least on threadpool level). I did this in a Draft PR #6607 and outlined a couple of shortcomings about this in there.

TLDR I'm convinced that we can offer worker_client functionality as is without a custom threadpool implementation, i.e. your problem 1.) and 2.) should go away without the need for a more high level change and we'd finally be able to update the threadpool to a newer cpython version


3.) lack of visibility is a fair point. Do you think it would be valuable if we showed these tasks on the dashboard somehow? How should something like this be made visible? I assume a decorator alone is not sufficient, is it? Right now these tasks just vanish from the dashboard. Not ideal but easy to fix

4.) is an actual problem. However, even with annotations/decorators/etc. we'd need to teach this to every custom executor, wouldn't we? Generally speaking, I'm OK if there are features that just work with distributed. That sounds sufficiently complicated that I wouldn't want to implement this for every executor

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

No branches or pull requests

2 participants