|
1 | 1 | import asyncio |
2 | | - |
3 | | -from distributed.core import rpc |
| 2 | +import aiohttp |
| 3 | +from contextlib import suppress |
4 | 4 |
|
5 | 5 | import kopf |
6 | 6 | import kubernetes_asyncio as kubernetes |
7 | 7 |
|
8 | 8 | from uuid import uuid4 |
9 | 9 |
|
| 10 | +from distributed.core import rpc |
| 11 | + |
10 | 12 | from dask_kubernetes.common.auth import ClusterAuth |
11 | 13 | from dask_kubernetes.common.networking import ( |
12 | 14 | get_scheduler_address, |
@@ -195,6 +197,52 @@ async def daskworkergroup_create(spec, name, namespace, logger, **kwargs): |
195 | 197 | ) |
196 | 198 |
|
197 | 199 |
|
| 200 | +async def retire_workers( |
| 201 | + n_workers, scheduler_service_name, worker_group_name, namespace, logger |
| 202 | +): |
| 203 | + # Try gracefully retiring via the HTTP API |
| 204 | + dashboard_address = await get_scheduler_address( |
| 205 | + scheduler_service_name, |
| 206 | + namespace, |
| 207 | + port_name="dashboard", |
| 208 | + ) |
| 209 | + async with aiohttp.ClientSession() as session: |
| 210 | + url = f"{dashboard_address}/api/v1/retire_workers" |
| 211 | + params = {"n": n_workers} |
| 212 | + async with session.post(url, json=params) as resp: |
| 213 | + if resp.status <= 300: |
| 214 | + retired_workers = await resp.json() |
| 215 | + return [retired_workers[w]["name"] for w in retired_workers.keys()] |
| 216 | + |
| 217 | + # Otherwise try gracefully retiring via the RPC |
| 218 | + logger.info( |
| 219 | + f"Scaling {worker_group_name} failed via the HTTP API, falling back to the Dask RPC" |
| 220 | + ) |
| 221 | + # Dask version mismatches between the operator and scheduler may cause this to fail in any number of unexpected ways |
| 222 | + with suppress(Exception): |
| 223 | + comm_address = await get_scheduler_address( |
| 224 | + scheduler_service_name, |
| 225 | + namespace, |
| 226 | + ) |
| 227 | + async with rpc(comm_address) as scheduler_comm: |
| 228 | + return await scheduler_comm.workers_to_close( |
| 229 | + n=n_workers, |
| 230 | + attribute="name", |
| 231 | + ) |
| 232 | + |
| 233 | + # Finally fall back to last-in-first-out scaling |
| 234 | + logger.info( |
| 235 | + f"Scaling {worker_group_name} failed via the Dask RPC, falling back to LIFO scaling" |
| 236 | + ) |
| 237 | + async with kubernetes.client.api_client.ApiClient() as api_client: |
| 238 | + api = kubernetes.client.CoreV1Api(api_client) |
| 239 | + workers = await api.list_namespaced_pod( |
| 240 | + namespace=namespace, |
| 241 | + label_selector=f"dask.org/workergroup-name={worker_group_name}", |
| 242 | + ) |
| 243 | + return [w["metadata"]["name"] for w in workers.items[:-n_workers]] |
| 244 | + |
| 245 | + |
198 | 246 | @kopf.on.update("daskworkergroup") |
199 | 247 | async def daskworkergroup_update(spec, name, namespace, logger, **kwargs): |
200 | 248 | async with kubernetes.client.api_client.ApiClient() as api_client: |
@@ -226,17 +274,13 @@ async def daskworkergroup_update(spec, name, namespace, logger, **kwargs): |
226 | 274 | f"Scaled worker group {name} up to {spec['worker']['replicas']} workers." |
227 | 275 | ) |
228 | 276 | if workers_needed < 0: |
229 | | - service_address = await get_scheduler_address( |
230 | | - f"{spec['cluster']}-service", namespace |
| 277 | + worker_ids = await retire_workers( |
| 278 | + n_workers=-workers_needed, |
| 279 | + scheduler_service_name=f"{spec['cluster']}-service", |
| 280 | + worker_group_name=name, |
| 281 | + namespace=namespace, |
| 282 | + logger=logger, |
231 | 283 | ) |
232 | | - logger.info( |
233 | | - f"Asking scheduler to retire {-workers_needed} on {service_address}" |
234 | | - ) |
235 | | - async with rpc(service_address) as scheduler: |
236 | | - worker_ids = await scheduler.workers_to_close( |
237 | | - n=-workers_needed, attribute="name" |
238 | | - ) |
239 | | - # TODO: Check that were deting workers in the right worker group |
240 | 284 | logger.info(f"Workers to close: {worker_ids}") |
241 | 285 | for wid in worker_ids: |
242 | 286 | await api.delete_namespaced_pod( |
|
0 commit comments