Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
23 changes: 23 additions & 0 deletions dask_kubernetes/operator/controller/controller.py
Original file line number Diff line number Diff line change
Expand Up @@ -481,6 +481,29 @@ async def get_desired_workers(scheduler_service_name, namespace, logger):
worker_group_scale_locks = defaultdict(lambda: asyncio.Lock())


@kopf.on.field("daskcluster.kubernetes.dask.org", field="spec.worker.replicas")
async def daskcluster_default_worker_group_replica_update(
name, namespace, meta, spec, old, new, body, logger, **kwargs
):
if old is None:
return
worker_group_name = f"{name}-default"

async with kubernetes.client.api_client.ApiClient() as api_client:
custom_objects_api = kubernetes.client.CustomObjectsApi(api_client)
custom_objects_api.api_client.set_default_header(
"content-type", "application/merge-patch+json"
)
await custom_objects_api.patch_namespaced_custom_object_scale(
group="kubernetes.dask.org",
version="v1",
plural="daskworkergroups",
namespace=namespace,
name=worker_group_name,
body={"spec": {"replicas": new}},
)
Comment on lines +492 to +504
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm excited about the kr8s changes in #696 because this will be hugely simplified.

Suggested change
async with kubernetes.client.api_client.ApiClient() as api_client:
custom_objects_api = kubernetes.client.CustomObjectsApi(api_client)
custom_objects_api.api_client.set_default_header(
"content-type", "application/merge-patch+json"
)
await custom_objects_api.patch_namespaced_custom_object_scale(
group="kubernetes.dask.org",
version="v1",
plural="daskworkergroups",
namespace=namespace,
name=worker_group_name,
body={"spec": {"replicas": new}},
)
worker_group = await DaskWorkerGroup.get(worker_group_name)
await worker_group.scale(new)

Happy to merge this as-is though and I'll update it in a follow-up once kr8s is integrated nicely.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, wow! Definitely looking forward to that!



@kopf.on.field("daskworkergroup.kubernetes.dask.org", field="spec.worker.replicas")
async def daskworkergroup_replica_update(
name, namespace, meta, spec, new, body, logger, **kwargs
Expand Down
42 changes: 42 additions & 0 deletions dask_kubernetes/operator/controller/tests/test_controller.py
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,48 @@ async def test_scalesimplecluster(k8s_cluster, kopf_runner, gen_cluster):
await client.wait_for_workers(3)


@pytest.mark.asyncio
async def test_scalesimplecluster_from_cluster_spec(
k8s_cluster, kopf_runner, gen_cluster
):
with kopf_runner as runner:
async with gen_cluster() as cluster_name:
scheduler_pod_name = "simple-scheduler"
worker_pod_name = "simple-default-worker"
service_name = "simple-scheduler"
while scheduler_pod_name not in k8s_cluster.kubectl("get", "pods"):
await asyncio.sleep(0.1)
while service_name not in k8s_cluster.kubectl("get", "svc"):
await asyncio.sleep(0.1)
while worker_pod_name not in k8s_cluster.kubectl("get", "pods"):
await asyncio.sleep(0.1)
k8s_cluster.kubectl(
"wait",
"pods",
"--for=condition=Ready",
scheduler_pod_name,
"--timeout=120s",
)
with k8s_cluster.port_forward(f"service/{service_name}", 8786) as port:
async with Client(
f"tcp://localhost:{port}", asynchronous=True
) as client:
k8s_cluster.kubectl(
"scale",
"--replicas=5",
"daskcluster.kubernetes.dask.org",
cluster_name,
)
await client.wait_for_workers(5)
k8s_cluster.kubectl(
"scale",
"--replicas=3",
"daskcluster.kubernetes.dask.org",
cluster_name,
)
await client.wait_for_workers(3)
Copy link
Member

@jacobtomlinson jacobtomlinson May 24, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This wont actually test anything because client.wait_for_workers(3) will just pass when there are 5 workers.

I opened dask/distributed#6377 a while ago to allow you to modify the behaviour for cases like this. I should nudge that PR along.

Maybe we should have a TODO comment here so that we come back and fix it up later.



@pytest.mark.timeout(180)
@pytest.mark.asyncio
async def test_simplecluster(k8s_cluster, kopf_runner, gen_cluster):
Expand Down