Skip to content

Commit

Permalink
Disable HTTP API by default (#6420)
Browse files Browse the repository at this point in the history
Disable HTTP API by default (#6420)
  • Loading branch information
jacobtomlinson authored May 23, 2022
1 parent d84485b commit 00a6445
Show file tree
Hide file tree
Showing 2 changed files with 41 additions and 5 deletions.
1 change: 0 additions & 1 deletion distributed/distributed.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
45 changes: 41 additions & 4 deletions distributed/http/scheduler/tests/test_scheduler_http.py
Original file line number Diff line number Diff line change
Expand Up @@ -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):
Expand Down Expand Up @@ -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(
Expand All @@ -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]}
Expand All @@ -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(
Expand All @@ -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(
Expand Down

0 comments on commit 00a6445

Please sign in to comment.