Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Disable HTTP API by default #6420

Merged
merged 3 commits into from
May 23, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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