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

Don't initialize mp_context on import #6580

Merged
merged 2 commits into from
Jun 16, 2022

Conversation

wence-
Copy link
Contributor

@wence- wence- commented Jun 15, 2022

  • Tests added / passed
  • Passes pre-commit run --all-files

The fork and spawn methods offered by multiprocessing interact badly with many infiniband (or other high performance) interconnects which don't support fork after the relevant low-level networking library has been initialised (or else crash in strange ways). Multiprocessing's forkserver method offers a way round this. As long as the forkserver is started (via multiprocessing.forkserver.ensure_running()) before networking library initialisation then things work (the child process that does the forking doesn't need the network library).

While one can control the method that distributed uses by setting DASK_DISTRIBUTED__WORKER__MULTIPROCESSING_METHOD=forkserver in the environment, it would be nice to be able to control this programmatically as well (using the "distributed.worker.multiprocessing-method" key). This PR enables that by deferring the context creation to function call time (at which point the distributed config can be inspected, and might already have been modified) rather than module import time. Since multiprocessing contexts are singleton objects, this should not be a performance hit.

Query: dask also provides a get_context method in dask.multiprocessing that respects a different config option ("multiprocessing.context"). The only difference is that the distributed version of the function adds a bunch of modules to the preload list for the forkserver case. Should I refactor to use the dask.multiprocessing version and handle preload there?

Allows controlling the context type through dask.config
programmatically, rather than needing to use environment variables, or
carefully control import order.
@GPUtester
Copy link
Collaborator

Can one of the admins verify this patch?

@fjetter
Copy link
Member

fjetter commented Jun 15, 2022

Thanks for your patch! Just skimming over the changes, this looks good to me. I am not sure if there was any deeper reason to having this setup at module import time. I guess having this cached should be sufficient.

I noticed you're having issues with our code linting (pre-commit hooks). See https://docs.dask.org/en/stable/develop.html#code-formatting for some guidance on how to set this up

Query: dask also provides a get_context method in dask.multiprocessing that respects a different config option ("multiprocessing.context"). The only difference is that the distributed version of the function adds a bunch of modules to the preload list for the forkserver case. Should I refactor to use the dask.multiprocessing version and handle preload there?

cc @jrbourbeau do you have any context about the dask.multiprocessing.get_context method?

@wence- wence- force-pushed the wence/feature/mp-get-context-singleton branch from e1270fc to 34f1bf0 Compare June 15, 2022 14:00
@wence-
Copy link
Contributor Author

wence- commented Jun 15, 2022

I noticed you're having issues with our code linting (pre-commit hooks). See https://docs.dask.org/en/stable/develop.html#code-formatting for some guidance on how to set this up

Sorry, I'd run pre-commit install but some PEBKAC stopped it firing, fixed.

@pentschev
Copy link
Member

add to allowlist

@github-actions
Copy link
Contributor

Unit Test Results

See test report for an extended history of previous test failures. This is useful for diagnosing flaky tests.

       15 files  ±0         15 suites  ±0   6h 20m 21s ⏱️ - 46m 44s
  2 869 tests +1    2 786 ✔️ +29    80 💤 ±0  1  - 27  2 🔥  - 1 
21 254 runs  +9  20 310 ✔️ +38  941 💤 +2  1  - 30  2 🔥  - 1 

For more details on these failures and errors, see this check.

Results for commit 34f1bf0. ± Comparison against base commit cb88e3b.

@fjetter
Copy link
Member

fjetter commented Jun 16, 2022

Should I refactor to use the dask.multiprocessing version and handle preload there?

I had another look at it. It may be a good idea to refactor all of this but the two config options have a slightly different meaning.
The dask.multiprocessing.get_context function is intended for the dask process backend w/out a distributed scheduler, e.g. when running dask.compute(graph, scheduler='processes'). However, this context method is controlling how we start distributed cluster worker processes.
I think if we started to mix these parameters, this would cause breaking changes for users and I don't see a huge benefit right now.

I'm not sure if the preload is worth it in dask. This optimization was mostly added for test runtime as noted in this comment and most dask tests should be using a threaded backend. Even distributed is by now using spawn as a default method (#3374)

@fjetter
Copy link
Member

fjetter commented Jun 16, 2022

Thank you @wence-

@fjetter fjetter merged commit 29dae02 into dask:main Jun 16, 2022
@wence- wence- deleted the wence/feature/mp-get-context-singleton branch June 16, 2022 10:53
@wence-
Copy link
Contributor Author

wence- commented Jun 16, 2022

Thanks!

rapids-bot bot pushed a commit to rapidsai/dask-cuda that referenced this pull request Jun 27, 2022
Allows selection of the method multiprocessing uses to start child
processes. Additionally, in the forkserver case, ensure the fork
server is up and running before any computation happens.

Potentially fixes #930. Needs dask/distributed#6580.

cc: @pentschev, @quasiben

Authors:
  - Lawrence Mitchell (https://github.com/wence-)

Approvers:
  - Peter Andreas Entschev (https://github.com/pentschev)

URL: #933
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.

4 participants