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

Add authentication to HTTP API #6431

Open
wants to merge 6 commits into
base: main
Choose a base branch
from
Open
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
4 changes: 4 additions & 0 deletions distributed/distributed-schema.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,10 @@ properties:
type: object
description: Settings for Dask's embedded HTTP Server
properties:
api-key:
type: string
description: |
API key required to access private HTTP API methods
routes:
type: array
description: |
Expand Down
2 changes: 2 additions & 0 deletions distributed/distributed.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -49,13 +49,15 @@ distributed:
lease-timeout: 30s # Maximum interval to wait for a Client refresh before a lease is invalidated and released.

http:
api-key: ""
routes:
- distributed.http.scheduler.prometheus
- distributed.http.scheduler.info
- distributed.http.scheduler.json
- distributed.http.health
- distributed.http.proxy
- distributed.http.statics
- distributed.http.scheduler.api

allowed-imports:
- dask
Expand Down
35 changes: 35 additions & 0 deletions distributed/http/scheduler/api.py
Original file line number Diff line number Diff line change
@@ -1,17 +1,52 @@
from __future__ import annotations

import asyncio
import functools
import json

import dask.config

from distributed.http.utils import RequestHandler


def require_auth(method_func):
@functools.wraps(method_func)
def wrapper(self):
auth = self.request.headers.get("Authorization", None)
key = dask.config.get("distributed.scheduler.http.api-key")
if key is False or (
key and auth and auth.startswith("Bearer ") and key == auth.split(" ")[-1]
):
if not asyncio.iscoroutinefunction(method_func):
return method_func(self)
else:

async def tmp():
return await method_func(self)

return tmp()
else:
self.set_status(403, "Unauthorized")
if not asyncio.iscoroutinefunction(method_func):
return
else:
# When wrapping a coroutine we need to return a coroutine even if it just returns None
async def tmp():
return

return tmp()

return wrapper


class APIHandler(RequestHandler):
def get(self):
self.write("API V1")
self.set_header("Content-Type", "text/plain")


class RetireWorkersHandler(RequestHandler):
@require_auth
async def post(self):
self.set_header("Content-Type", "application/json")
scheduler = self.server
Expand Down
106 changes: 73 additions & 33 deletions distributed/http/scheduler/tests/test_scheduler_http.py
Original file line number Diff line number Diff line change
Expand Up @@ -251,60 +251,107 @@ async def test_eventstream(c, s, a, b):
ws_client.close()


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})
async def test_api(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 == 200
assert resp.headers["Content-Type"] == "text/plain"
assert (await resp.text()) == "API V1"


@gen_cluster(client=True, clean_kwargs={"threads": False})
async def test_api_auth_defaults(c, s, a, b):
async with aiohttp.ClientSession() as session:
url = f"http://localhost:{s.http_server.port}/api/v1/retire_workers"
params = {"workers": [a.address, b.address]}

async with session.post(url, json=params) as resp:
assert resp.status == 403


@gen_cluster(
client=True,
clean_kwargs={"threads": False},
config={
"distributed.scheduler.http.routes": DEFAULT_ROUTES
+ ["distributed.http.scheduler.api"]
"distributed.scheduler.http.api-key": "abc123",
},
)
async def test_api(c, s, a, b):
async def test_api_auth(c, s, a, b):
async with aiohttp.ClientSession() as session:
async with session.get(
"http://localhost:%d/api/v1" % s.http_server.port
url = f"http://localhost:{s.http_server.port}/api/v1/retire_workers"
params = {"workers": [a.address, b.address]}

async with session.post(url, json=params) as resp:
assert resp.status == 403

async with session.post(
url, json=params, headers={"Authorization": "Bearer foobarbaz"}
) as resp:
assert resp.status == 403

async with session.post(
url, json=params, headers={"Authorization": "Bearer abc"}
) as resp:
assert resp.status == 403

async with session.post(
url, json=params, headers={"Authorization": "Bearer "}
) as resp:
assert resp.status == 403

async with session.post(
url, json=params, headers={"Authorization": "Bearer abc123456"}
) as resp:
assert resp.status == 403

async with session.post(
url, json=params, headers={"Authorization": "Bearer abc123"}
) as resp:
assert resp.status == 200
assert resp.headers["Content-Type"] == "text/plain"
assert (await resp.text()) == "API V1"


@gen_cluster(
client=True,
clean_kwargs={"threads": False},
config={
"distributed.scheduler.http.routes": DEFAULT_ROUTES
+ ["distributed.http.scheduler.api"]
"distributed.scheduler.http.api-key": False,
},
)
async def test_retire_workers(c, s, a, b):
async def test_api_auth_disabled(c, s, a, b):
async with aiohttp.ClientSession() as session:
url = f"http://localhost:{s.http_server.port}/api/v1/retire_workers"
params = {"workers": [a.address, b.address]}
async with session.post(
"http://localhost:%d/api/v1/retire_workers" % s.http_server.port,
json=params,
) as resp:

async with session.post(url, json=params) as resp:
assert resp.status == 200
assert resp.headers["Content-Type"] == "application/json"
retired_workers_info = json.loads(await resp.text())
assert len(retired_workers_info) == 2


@gen_cluster(
client=True,
clean_kwargs={"threads": False},
config={
"distributed.scheduler.http.routes": DEFAULT_ROUTES
+ ["distributed.http.scheduler.api"]
"distributed.scheduler.http.api-key": "abc123",
},
)
async def test_get_workers(c, s, a, b):
async def test_api_retire_workers(c, s, a, b):
async with aiohttp.ClientSession() as session:
url = f"http://localhost:{s.http_server.port}/api/v1/retire_workers"
params = {"workers": [a.address, b.address]}

async with session.post(
url, json=params, headers={"Authorization": "Bearer abc123"}
) as resp:
assert resp.status == 200
assert resp.headers["Content-Type"] == "application/json"
retired_workers_info = json.loads(await resp.text())
assert len(retired_workers_info) == 2


@gen_cluster(client=True, clean_kwargs={"threads": False})
async def test_api_get_workers(c, s, a, b):
async with aiohttp.ClientSession() as session:
async with session.get(
"http://localhost:%d/api/v1/get_workers" % s.http_server.port
Expand All @@ -316,15 +363,8 @@ 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},
config={
"distributed.scheduler.http.routes": DEFAULT_ROUTES
+ ["distributed.http.scheduler.api"]
},
)
async def test_adaptive_target(c, s, a, b):
@gen_cluster(client=True, clean_kwargs={"threads": False})
async def test_api_adaptive_target(c, s, a, b):
async with aiohttp.ClientSession() as session:
async with session.get(
"http://localhost:%d/api/v1/adaptive_target" % s.http_server.port
Expand Down
15 changes: 13 additions & 2 deletions docs/source/http_services.rst
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ Scheduler API

Scheduler methods exposed by the API with an example of the request body they take

- ``/api/v1/retire_workers`` : retire certain workers on the scheduler
- ``/api/v1/retire_workers`` : retire certain workers on the scheduler (requires auth)

.. code-block:: json

Expand All @@ -63,7 +63,18 @@ 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

.. note::
API methods that modify the state of the scheduler require an API key to be set in the ``Authorization`` header.
This API key can be set via ``distributed.scheduler.http.api-key`` in the Dask config.

.. code-block:: console

$ curl -H "Authorization: Bearer {api-key}" http://localhost:8787/api/v1/retire_workers

.. warning::
API authentication can be disabled by setting ``distributed.scheduler.http.api-key`` to ``False`` but this is not recommended.

Individual bokeh plots
----------------------
Expand Down