diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 942eab8ff9a..dda31a0d2d2 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -246,7 +246,15 @@ properties: These modules will have a ``routes`` keyword that gets added to the main HTTP Server. This is also a list that can be extended with user defined modules. + These routes should be read-only; they should not be able to affect cluster state. + insecure-routes: + type: array + description: | + A list of modules that will only be added as HTTP routes if unauthenticated client connections are allowed + These routes offer the ability to affect cluster state. Unlike client connections, HTTP routes do not support + authentication. Since the client is the primary way to affect cluster state, it's assumed that if client + connections require authentication, these routes should not be exposed. allowed-imports: type: array description: | diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index 931c7438ad2..8073d7713d1 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -53,10 +53,11 @@ 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 + insecure-routes: + - distributed.http.scheduler.api allowed-imports: - dask diff --git a/distributed/http/scheduler/tests/test_scheduler_http.py b/distributed/http/scheduler/tests/test_scheduler_http.py index 94f7563039e..ee3584b796c 100644 --- a/distributed/http/scheduler/tests/test_scheduler_http.py +++ b/distributed/http/scheduler/tests/test_scheduler_http.py @@ -13,7 +13,7 @@ from dask.sizeof import sizeof from distributed.utils import is_valid_xml -from distributed.utils_test import gen_cluster, inc, slowinc +from distributed.utils_test import gen_cluster, inc, slowinc, tls_only_security @gen_cluster(client=True) @@ -259,6 +259,15 @@ async def test_api(c, s, a, b): assert (await resp.text()) == "API V1" +@gen_cluster(client=True, clean_kwargs={"threads": False}, security=tls_only_security()) +async def test_api_disabled_if_secure(c, s, a, b): + async with aiohttp.ClientSession() as session: + async with session.get( + "http://localhost:%d/api/v1" % s.http_server.port + ) as resp: + assert resp.status == 404 + + @gen_cluster(client=True, clean_kwargs={"threads": False}) async def test_retire_workers(c, s, a, b): async with aiohttp.ClientSession() as session: diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 45d7c58bf79..9617afa7bcc 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -2951,6 +2951,10 @@ def __init__( except ImportError: show_dashboard = False http_server_modules.append("distributed.http.scheduler.missing_bokeh") + if not self.security.require_encryption: + http_server_modules.extend( + dask.config.get("distributed.scheduler.http.insecure-routes") + ) routes = get_handlers( server=self, modules=http_server_modules, prefix=http_prefix ) diff --git a/docs/source/http_services.rst b/docs/source/http_services.rst index 31bb62292a7..a4caf906c39 100644 --- a/docs/source/http_services.rst +++ b/docs/source/http_services.rst @@ -52,7 +52,11 @@ Pages and JSON endpoints served by the scheduler Scheduler API ------------- -Scheduler methods exposed by the API with an example of the request body they take +Scheduler methods exposed by the API with an example of the request body they take. + +.. note:: + To prevent unauthorized access, the scheduler API is disabled by default if `tls`_ is enabled. + See the ``distributed.http.insecure-routes`` :doc:`config ` setting. - ``/api/v1/retire_workers`` : retire certain workers on the scheduler @@ -63,7 +67,7 @@ Scheduler methods exposed by the API with an example of the request body they ta } - ``/api/v1/get_workers`` : get all workers on the scheduler -- ``/api/v1/adaptive_target`` : get the target number of workers based on the scheduler's load +- ``/api/v1/adaptive_target`` : get the target number of workers based on the scheduler's load Individual bokeh plots ----------------------