Skip to content

Commit

Permalink
Remove OMP_NUM_THREADS if set to empty
Browse files Browse the repository at this point in the history
Generally though, sometimes our use of OMP_NUM_THREADS makes other
systems sad.  It's unfortunately somewhat difficult to turn off.  A
common approach is that people set `OMP_NUM_THREADS=""` but this doesn't
properly unset things.

I'm curious if an approach like this would be helpful.

There might be a cleaner way to do this, and this might be a bad idea.
Please feel free to reject.  It was just easy to put this up as a PR.
  • Loading branch information
mrocklin committed Oct 29, 2024
1 parent 2953090 commit e8d9b81
Show file tree
Hide file tree
Showing 2 changed files with 19 additions and 0 deletions.
6 changes: 6 additions & 0 deletions distributed/nanny.py
Original file line number Diff line number Diff line change
Expand Up @@ -240,6 +240,7 @@ def __init__( # type: ignore[no-untyped-def]
# https://github.com/dask/dask/issues/6640.
self.pre_spawn_env.update({"PYTHONHASHSEED": "6640"})


self.env = merge(
self.pre_spawn_env,
_get_env_variables("distributed.nanny.environ"),
Expand Down Expand Up @@ -1031,4 +1032,9 @@ def _get_env_variables(config_key: str) -> dict[str, str]:
# Override dask config with explicitly defined env variables from the OS
# Allow unsetting a variable in a config override by setting its value to None.
cfg = {k: os.environ.get(k, str(v)) for k, v in cfg.items() if v is not None}

for k, v in list(cfg.items()):
if "_NUM_THREADS" in k and not v:
del cfg[k]

return cfg
13 changes: 13 additions & 0 deletions distributed/tests/test_nanny.py
Original file line number Diff line number Diff line change
Expand Up @@ -312,6 +312,19 @@ async def test_environment_variable(c, s):
await asyncio.gather(a.close(), b.close())


@gen_cluster(
nthreads=[("", 1)],
client=True,
Worker=Nanny,
config={
"distributed.nanny.pre-spawn-environ": {"OMP_NUM_THREADS": ""},
},
)
async def test_omp_num_threads_off(c, s, a):
results = await c.run(lambda: "OMP_NUM_THREADS" in os.environ)
assert results == {a.worker_address: False}


@gen_cluster(nthreads=[], client=True)
async def test_environment_variable_by_config(c, s, monkeypatch):
with dask.config.set({"distributed.nanny.environ": "456"}):
Expand Down

0 comments on commit e8d9b81

Please sign in to comment.