Skip to content

Commit 1efdc51

Browse files
jo-migoJohanna Goergen
andauthored
Validate Dask Cluster Names (#871)
* fix(validation): Validate Dask Cluster Names This commit introduces cluster name validation in order to avoid the invalid state in which a `DaskCluster` resource with a too-long or RFC-1123-noncompliant name is created but cannot be deleted while the operator retries infinitely to create a scheduler service (see #826 for more details on this bug). Issues fixed: #870 #826 * Actually, stop removing the dask cluster automatically. It can be manually deleted. * Move the cluster name validation into a common module, add it to KubeCluster init, and add tests --------- Co-authored-by: Johanna Goergen <[email protected]>
1 parent a7cbd74 commit 1efdc51

File tree

8 files changed

+153
-11
lines changed

8 files changed

+153
-11
lines changed

dask_kubernetes/common/objects.py

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,14 +1,20 @@
11
"""
22
Convenience functions for creating pod templates.
33
"""
4+
45
import copy
56
import json
67
from collections import namedtuple
78

89
from kubernetes import client
910
from kubernetes.client.configuration import Configuration
1011

11-
from dask_kubernetes.constants import KUBECLUSTER_CONTAINER_NAME
12+
from dask_kubernetes.constants import (
13+
KUBECLUSTER_CONTAINER_NAME,
14+
MAX_CLUSTER_NAME_LEN,
15+
VALID_CLUSTER_NAME,
16+
)
17+
from dask_kubernetes.exceptions import ValidationError
1218

1319
_FakeResponse = namedtuple("_FakeResponse", ["data"])
1420

@@ -365,3 +371,16 @@ def clean_pdb_template(pdb_template):
365371
pdb_template.spec.selector = client.V1LabelSelector()
366372

367373
return pdb_template
374+
375+
376+
def validate_cluster_name(cluster_name: str) -> None:
377+
"""Raise exception if cluster name is too long and/or has invalid characters"""
378+
if not VALID_CLUSTER_NAME.match(cluster_name):
379+
raise ValidationError(
380+
message=(
381+
f"The DaskCluster {cluster_name} is invalid: a lowercase RFC 1123 subdomain must "
382+
"consist of lower case alphanumeric characters, '-' or '.', and must start "
383+
"and end with an alphanumeric character. DaskCluster name must also be under "
384+
f"{MAX_CLUSTER_NAME_LEN} characters."
385+
)
386+
)

dask_kubernetes/common/tests/test_objects.py

Lines changed: 24 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,8 @@
1-
from dask_kubernetes.common.objects import make_pod_from_dict
2-
from dask_kubernetes.constants import KUBECLUSTER_CONTAINER_NAME
1+
import pytest
2+
3+
from dask_kubernetes.common.objects import make_pod_from_dict, validate_cluster_name
4+
from dask_kubernetes.constants import KUBECLUSTER_CONTAINER_NAME, MAX_CLUSTER_NAME_LEN
5+
from dask_kubernetes.exceptions import ValidationError
36

47

58
def test_make_pod_from_dict():
@@ -64,3 +67,22 @@ def test_make_pod_from_dict_default_container_name():
6467
assert pod.spec.containers[0].name == "dask-0"
6568
assert pod.spec.containers[1].name == "sidecar"
6669
assert pod.spec.containers[2].name == "dask-2"
70+
71+
72+
@pytest.mark.parametrize(
73+
"cluster_name",
74+
[
75+
(MAX_CLUSTER_NAME_LEN + 1) * "a",
76+
"invalid.chars.in.name",
77+
],
78+
)
79+
def test_validate_cluster_name_raises_on_invalid_name(
80+
cluster_name,
81+
):
82+
83+
with pytest.raises(ValidationError):
84+
validate_cluster_name(cluster_name)
85+
86+
87+
def test_validate_cluster_name_success_on_valid_name():
88+
assert validate_cluster_name("valid-cluster-name-123") is None

dask_kubernetes/constants.py

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1 +1,11 @@
1+
import re
2+
13
KUBECLUSTER_CONTAINER_NAME = "dask-container"
4+
KUBERNETES_MAX_RESOURCE_NAME_LENGTH = 63
5+
SCHEDULER_NAME_TEMPLATE = "{cluster_name}-scheduler"
6+
MAX_CLUSTER_NAME_LEN = KUBERNETES_MAX_RESOURCE_NAME_LENGTH - len(
7+
SCHEDULER_NAME_TEMPLATE.format(cluster_name="")
8+
)
9+
VALID_CLUSTER_NAME = re.compile(
10+
rf"^(?=.{{,{MAX_CLUSTER_NAME_LEN}}}$)[a-z0-9]([-a-z0-9]*[a-z0-9])?$"
11+
)

dask_kubernetes/exceptions.py

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4,3 +4,10 @@ class CrashLoopBackOffError(Exception):
44

55
class SchedulerStartupError(Exception):
66
"""Scheduler failed to start."""
7+
8+
9+
class ValidationError(Exception):
10+
"""Manifest validation exception"""
11+
12+
def __init__(self, message: str) -> None:
13+
self.message = message

dask_kubernetes/operator/controller/controller.py

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,9 @@
1515
from importlib_metadata import entry_points
1616
from kr8s.asyncio.objects import Deployment, Pod, Service
1717

18+
from dask_kubernetes.common.objects import validate_cluster_name
19+
from dask_kubernetes.constants import SCHEDULER_NAME_TEMPLATE
20+
from dask_kubernetes.exceptions import ValidationError
1821
from dask_kubernetes.operator._objects import (
1922
DaskAutoscaler,
2023
DaskCluster,
@@ -76,7 +79,7 @@ def build_scheduler_deployment_spec(
7679
}
7780
)
7881
metadata = {
79-
"name": f"{cluster_name}-scheduler",
82+
"name": SCHEDULER_NAME_TEMPLATE.format(cluster_name=cluster_name),
8083
"labels": labels,
8184
"annotations": annotations,
8285
}
@@ -109,7 +112,7 @@ def build_scheduler_service_spec(cluster_name, spec, annotations, labels):
109112
"apiVersion": "v1",
110113
"kind": "Service",
111114
"metadata": {
112-
"name": f"{cluster_name}-scheduler",
115+
"name": SCHEDULER_NAME_TEMPLATE.format(cluster_name=cluster_name),
113116
"labels": labels,
114117
"annotations": annotations,
115118
},
@@ -274,6 +277,12 @@ async def daskcluster_create(name, namespace, logger, patch, **kwargs):
274277
This allows us to track that the operator is running.
275278
"""
276279
logger.info(f"DaskCluster {name} created in {namespace}.")
280+
try:
281+
validate_cluster_name(name)
282+
except ValidationError as validation_exc:
283+
patch.status["phase"] = "Error"
284+
raise kopf.PermanentError(validation_exc.message)
285+
277286
patch.status["phase"] = "Created"
278287

279288

@@ -600,7 +609,7 @@ async def daskworkergroup_replica_update(
600609
if workers_needed < 0:
601610
worker_ids = await retire_workers(
602611
n_workers=-workers_needed,
603-
scheduler_service_name=f"{cluster_name}-scheduler",
612+
scheduler_service_name=SCHEDULER_NAME_TEMPLATE.format(cluster_name),
604613
worker_group_name=name,
605614
namespace=namespace,
606615
logger=logger,

dask_kubernetes/operator/controller/tests/test_controller.py

Lines changed: 59 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@
1111
from dask.distributed import Client
1212
from kr8s.asyncio.objects import Deployment, Pod, Service
1313

14+
from dask_kubernetes.constants import MAX_CLUSTER_NAME_LEN
1415
from dask_kubernetes.operator._objects import DaskCluster, DaskJob, DaskWorkerGroup
1516
from dask_kubernetes.operator.controller import (
1617
KUBERNETES_DATETIME_FORMAT,
@@ -22,17 +23,32 @@
2223

2324
_EXPECTED_ANNOTATIONS = {"test-annotation": "annotation-value"}
2425
_EXPECTED_LABELS = {"test-label": "label-value"}
26+
DEFAULT_CLUSTER_NAME = "simple"
2527

2628

2729
@pytest.fixture()
28-
def gen_cluster(k8s_cluster, ns):
30+
def gen_cluster_manifest(tmp_path):
31+
def factory(cluster_name=DEFAULT_CLUSTER_NAME):
32+
original_manifest_path = os.path.join(DIR, "resources", "simplecluster.yaml")
33+
with open(original_manifest_path, "r") as original_manifest_file:
34+
manifest = yaml.safe_load(original_manifest_file)
35+
36+
manifest["metadata"]["name"] = cluster_name
37+
new_manifest_path = tmp_path / "cluster.yaml"
38+
new_manifest_path.write_text(yaml.safe_dump(manifest))
39+
return tmp_path
40+
41+
return factory
42+
43+
44+
@pytest.fixture()
45+
def gen_cluster(k8s_cluster, ns, gen_cluster_manifest):
2946
"""Yields an instantiated context manager for creating/deleting a simple cluster."""
3047

3148
@asynccontextmanager
32-
async def cm():
33-
cluster_path = os.path.join(DIR, "resources", "simplecluster.yaml")
34-
cluster_name = "simple"
49+
async def cm(cluster_name=DEFAULT_CLUSTER_NAME):
3550

51+
cluster_path = gen_cluster_manifest(cluster_name)
3652
# Create cluster resource
3753
k8s_cluster.kubectl("apply", "-n", ns, "-f", cluster_path)
3854
while cluster_name not in k8s_cluster.kubectl(
@@ -695,3 +711,42 @@ async def test_object_dask_job(k8s_cluster, kopf_runner, gen_job):
695711

696712
cluster = await job.cluster()
697713
assert isinstance(cluster, DaskCluster)
714+
715+
716+
async def _get_cluster_status(k8s_cluster, ns, cluster_name):
717+
"""
718+
Will loop infinitely in search of non-falsey cluster status.
719+
Make sure there is a timeout on any test which calls this.
720+
"""
721+
while True:
722+
cluster_status = k8s_cluster.kubectl(
723+
"get",
724+
"-n",
725+
ns,
726+
"daskcluster.kubernetes.dask.org",
727+
cluster_name,
728+
"-o",
729+
"jsonpath='{.status.phase}'",
730+
).strip("'")
731+
if cluster_status:
732+
return cluster_status
733+
await asyncio.sleep(0.1)
734+
735+
736+
@pytest.mark.timeout(180)
737+
@pytest.mark.anyio
738+
@pytest.mark.parametrize(
739+
"cluster_name,expected_status",
740+
[
741+
("valid-name", "Created"),
742+
((MAX_CLUSTER_NAME_LEN + 1) * "a", "Error"),
743+
("invalid.chars.in.name", "Error"),
744+
],
745+
)
746+
async def test_create_cluster_validates_name(
747+
cluster_name, expected_status, k8s_cluster, kopf_runner, gen_cluster
748+
):
749+
with kopf_runner:
750+
async with gen_cluster(cluster_name=cluster_name) as (_, ns):
751+
actual_status = await _get_cluster_status(k8s_cluster, ns, cluster_name)
752+
assert expected_status == actual_status

dask_kubernetes/operator/kubecluster/kubecluster.py

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
from rich.table import Table
3030
from tornado.ioloop import IOLoop
3131

32+
from dask_kubernetes.common.objects import validate_cluster_name
3233
from dask_kubernetes.exceptions import CrashLoopBackOffError, SchedulerStartupError
3334
from dask_kubernetes.operator._objects import (
3435
DaskAutoscaler,
@@ -258,6 +259,7 @@ def __init__(
258259
name = name.format(
259260
user=getpass.getuser(), uuid=str(uuid.uuid4())[:10], **os.environ
260261
)
262+
validate_cluster_name(name)
261263
self._instances.add(self)
262264
self._rich_spinner = Spinner("dots", speed=0.5)
263265
self._startup_component_status: dict = {}

dask_kubernetes/operator/kubecluster/tests/test_kubecluster.py

Lines changed: 19 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,8 @@
22
from dask.distributed import Client
33
from distributed.utils import TimeoutError
44

5-
from dask_kubernetes.exceptions import SchedulerStartupError
5+
from dask_kubernetes.constants import MAX_CLUSTER_NAME_LEN
6+
from dask_kubernetes.exceptions import SchedulerStartupError, ValidationError
67
from dask_kubernetes.operator import KubeCluster, make_cluster_spec
78

89

@@ -202,3 +203,20 @@ def test_typo_resource_limits(ns):
202203
},
203204
namespace=ns,
204205
)
206+
207+
208+
@pytest.mark.parametrize(
209+
"cluster_name",
210+
[
211+
(MAX_CLUSTER_NAME_LEN + 1) * "a",
212+
"invalid.chars.in.name",
213+
],
214+
)
215+
def test_invalid_cluster_name_fails(cluster_name, kopf_runner, docker_image, ns):
216+
with kopf_runner:
217+
with pytest.raises(ValidationError):
218+
KubeCluster(
219+
name=cluster_name,
220+
namespace=ns,
221+
image=docker_image,
222+
)

0 commit comments

Comments
 (0)