diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index 931c7438ad2..74d59addb35 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -53,7 +53,6 @@ distributed: - distributed.http.scheduler.prometheus - distributed.http.scheduler.info - distributed.http.scheduler.json - - distributed.http.scheduler.api - distributed.http.health - distributed.http.proxy - distributed.http.statics diff --git a/distributed/http/scheduler/tests/test_scheduler_http.py b/distributed/http/scheduler/tests/test_scheduler_http.py index 94f7563039e..e86f004b47e 100644 --- a/distributed/http/scheduler/tests/test_scheduler_http.py +++ b/distributed/http/scheduler/tests/test_scheduler_http.py @@ -10,11 +10,14 @@ from tornado.escape import url_escape from tornado.httpclient import AsyncHTTPClient, HTTPClientError +import dask.config from dask.sizeof import sizeof from distributed.utils import is_valid_xml from distributed.utils_test import gen_cluster, inc, slowinc +DEFAULT_ROUTES = dask.config.get("distributed.scheduler.http.routes") + @gen_cluster(client=True) async def test_connect(c, s, a, b): @@ -248,7 +251,20 @@ async def test_eventstream(c, s, a, b): ws_client.close() -@gen_cluster(client=True, clean_kwargs={"threads": False}) +def test_api_disabled_by_default(): + assert "distributed.http.scheduler.api" not in dask.config.get( + "distributed.scheduler.http.routes" + ) + + +@gen_cluster( + client=True, + clean_kwargs={"threads": False}, + config={ + "distributed.scheduler.http.routes": DEFAULT_ROUTES + + ["distributed.http.scheduler.api"] + }, +) async def test_api(c, s, a, b): async with aiohttp.ClientSession() as session: async with session.get( @@ -259,7 +275,14 @@ async def test_api(c, s, a, b): assert (await resp.text()) == "API V1" -@gen_cluster(client=True, clean_kwargs={"threads": False}) +@gen_cluster( + client=True, + clean_kwargs={"threads": False}, + config={ + "distributed.scheduler.http.routes": DEFAULT_ROUTES + + ["distributed.http.scheduler.api"] + }, +) async def test_retire_workers(c, s, a, b): async with aiohttp.ClientSession() as session: params = {"workers": [a.address, b.address]} @@ -273,7 +296,14 @@ async def test_retire_workers(c, s, a, b): assert len(retired_workers_info) == 2 -@gen_cluster(client=True, clean_kwargs={"threads": False}) +@gen_cluster( + client=True, + clean_kwargs={"threads": False}, + config={ + "distributed.scheduler.http.routes": DEFAULT_ROUTES + + ["distributed.http.scheduler.api"] + }, +) async def test_get_workers(c, s, a, b): async with aiohttp.ClientSession() as session: async with session.get( @@ -286,7 +316,14 @@ async def test_get_workers(c, s, a, b): assert set(workers_address) == {a.address, b.address} -@gen_cluster(client=True, clean_kwargs={"threads": False}) +@gen_cluster( + client=True, + clean_kwargs={"threads": False}, + config={ + "distributed.scheduler.http.routes": DEFAULT_ROUTES + + ["distributed.http.scheduler.api"] + }, +) async def test_adaptive_target(c, s, a, b): async with aiohttp.ClientSession() as session: async with session.get(