Skip to content

Commit

Permalink
Support for unsetting environment variables
Browse files Browse the repository at this point in the history
  • Loading branch information
crusaderky committed Sep 4, 2023
1 parent 9ba2eed commit 702f065
Show file tree
Hide file tree
Showing 3 changed files with 46 additions and 4 deletions.
13 changes: 10 additions & 3 deletions distributed/distributed-schema.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -644,15 +644,22 @@ properties:
Environment variables to set on all worker processes started by nannies.
These variables are set in the worker process after it has started.
To unset a variable in a config override, set its value to null (if using
a YAML config file) or None (if using dask.config.set) or "None" (if using
a DASK_* environment variable).
pre-spawn-environ:
type: object
description: |
Environment variables to set on all worker processes started by nannies.
These variables are set within the Nanny process,
before spawning the worker process.
Should be used for variables that must be set before
These variables are set within the Nanny process, before spawning the
worker process. Should be used for variables that must be set before
process startup, interpreter startup, or imports.
To unset a variable in a config override, set its value to null (if using
a YAML config file) or None (if using dask.config.set) or "None" (if using
a DASK_* environment variable).
client:
type: object
description: |
Expand Down
3 changes: 2 additions & 1 deletion distributed/nanny.py
Original file line number Diff line number Diff line change
Expand Up @@ -1008,5 +1008,6 @@ def _get_env_variables(config_key: str) -> dict[str, str]:
f"{config_key} configuration must be of type dict. Instead got {type(cfg)}"
)
# Override dask config with explicitly defined env variables from the OS
cfg = {k: os.environ.get(k, str(v)) for k, v in cfg.items()}
# 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}

Check warning on line 1012 in distributed/nanny.py

View check run for this annotation

Codecov / codecov/patch

distributed/nanny.py#L1012

Added line #L1012 was not covered by tests
return cfg
34 changes: 34 additions & 0 deletions distributed/tests/test_nanny.py
Original file line number Diff line number Diff line change
Expand Up @@ -349,6 +349,40 @@ async def test_environment_variable_pre_post_spawn(c, s, n):
assert "POST-SPAWN" not in os.environ


@gen_cluster(
nthreads=[],
client=True,
config={
"distributed.nanny.pre-spawn-environ.PRE1": 1,
"distributed.nanny.pre-spawn-environ.PRE2": 2,
"distributed.nanny.pre-spawn-environ.PRE3": 3,
"distributed.nanny.environ.POST1": 4,
"distributed.nanny.environ.POST2": 5,
"distributed.nanny.environ.POST3": 6,
},
)
async def test_environment_variable_overlay(c, s):
"""You can set a value to None to unset a variable in a config overlay"""
# Not the same as running Nanny(config=...), which would not work for pre-spawn
# variables
with dask.config.set(
{
"distributed.nanny.pre-spawn-environ.PRE2": 7,
"distributed.nanny.pre-spawn-environ.PRE3": None,
"distributed.nanny.environ.POST2": 8,
"distributed.nanny.environ.POST3": None,
},
):
async with Nanny(s.address):
env = await c.submit(lambda: os.environ)
assert env["PRE1"] == "1"
assert env["PRE2"] == "7"
assert "PRE3" not in env
assert env["POST1"] == "4"
assert env["POST2"] == "8"
assert "POST3" not in env


@gen_cluster(client=True, nthreads=[])
async def test_config_param_overlays(c, s):
with dask.config.set({"test123.foo": 1, "test123.bar": 2}):
Expand Down

0 comments on commit 702f065

Please sign in to comment.