Skip to content

Commit

Permalink
Warn for deprecations on dask.config.set() (dask#8179)
Browse files Browse the repository at this point in the history
  • Loading branch information
crusaderky authored Sep 14, 2023
1 parent e57d1c5 commit b75f5da
Show file tree
Hide file tree
Showing 3 changed files with 46 additions and 3 deletions.
8 changes: 6 additions & 2 deletions distributed/config.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@

dask.config.update_defaults(defaults)

aliases = {
deprecations = {
"allowed-failures": "distributed.scheduler.allowed-failures",
"bandwidth": "distributed.scheduler.bandwidth",
"default-data-size": "distributed.scheduler.default-data-size",
Expand Down Expand Up @@ -55,7 +55,11 @@
"rmm": "distributed.rmm",
}

dask.config.rename(aliases)
# Affects yaml and env variables configs, as well as calls to dask.config.set()
# before importing distributed
dask.config.rename(deprecations)
# Affects dask.config.set() from now on
dask.config.deprecations.update(deprecations)


#########################
Expand Down
39 changes: 39 additions & 0 deletions distributed/tests/test_config.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@
import pytest
import yaml

import dask.config

from distributed.config import initialize_logging
from distributed.utils_test import captured_handler, new_config, new_config_file

Expand Down Expand Up @@ -360,3 +362,40 @@ def test_uvloop_event_loop():
[sys.executable, "-c", script],
env={"DASK_DISTRIBUTED__ADMIN__EVENT_LOOP": "uvloop"},
)


@pytest.mark.parametrize(
"args,kwargs",
[
((), {"allowed_failures": 123}),
(({"allowed_failures": 123},), {}),
(({"allowed-failures": 123},), {}),
],
)
def test_deprecations_on_set(args, kwargs):
with pytest.warns(FutureWarning) as info:
with dask.config.set(*args, **kwargs):
assert dask.config.get("distributed.scheduler.allowed-failures") == 123

assert "distributed.scheduler.allowed-failures" in str(info[0].message)


def test_deprecations_on_env_variables(monkeypatch):
d = {}
monkeypatch.setenv("DASK_ALLOWED_FAILURES", "123")
with pytest.warns(FutureWarning) as info:
dask.config.refresh(config=d)
assert "distributed.scheduler.allowed-failures" in str(info[0].message)
assert dask.config.get("distributed.scheduler.allowed-failures", config=d) == 123


@pytest.mark.parametrize("key", ["allowed-failures", "allowed_failures"])
def test_deprecations_on_yaml(tmp_path, key):
d = {}
with open(tmp_path / "dask.yaml", "w") as fh:
yaml.dump({key: 123}, fh)

with pytest.warns(FutureWarning) as info:
dask.config.refresh(config=d, paths=[tmp_path])
assert "distributed.scheduler.allowed-failures" in str(info[0].message)
assert dask.config.get("distributed.scheduler.allowed-failures", config=d) == 123
2 changes: 1 addition & 1 deletion distributed/tests/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -1051,7 +1051,7 @@ async def test_tick_logging(s, a, b):
@pytest.mark.parametrize("serialize", [echo_serialize, echo_no_serialize])
@gen_test()
async def test_compression(compression, serialize):
with dask.config.set(compression=compression):
with dask.config.set({"distributed.comm.compression": compression}):
async with Server({"echo": serialize}) as server:
await server.listen("tcp://")

Expand Down

0 comments on commit b75f5da

Please sign in to comment.