1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287
|
from __future__ import annotations
import abc
import asyncio
import copy
import json
import logging
import math
import os
import sys
import tempfile
import uuid
from pathlib import Path
from typing import Any
import psutil
import toolz
from dask.system import CPU_COUNT
from distributed.compatibility import WINDOWS
from distributed.deploy.spec import ProcessInterface, SpecCluster
from distributed.deploy.utils import nprocesses_nthreads
from distributed.utils import Deadline
from distributed.worker_memory import parse_memory_limit
logger = logging.getLogger(__name__)
class Subprocess(ProcessInterface, abc.ABC):
process: asyncio.subprocess.Process | None
def __init__(self):
if WINDOWS:
# FIXME: distributed#7434
raise RuntimeError("Subprocess does not support Windows.")
self.process = None
super().__init__()
async def start(self) -> None:
await self._start()
await super().start()
@abc.abstractmethod
async def _start(self) -> None:
"""Start the subprocess"""
async def close(self) -> None:
if self.process and self.process.returncode is None:
for child in psutil.Process(self.process.pid).children(recursive=True):
child.kill()
self.process.kill()
await self.process.communicate()
self.process = None
await super().close()
class SubprocessScheduler(Subprocess):
"""A local Dask scheduler running in a dedicated subprocess
Parameters
----------
scheduler_kwargs:
Keywords to pass on to the ``Scheduler`` class constructor
"""
scheduler_kwargs: dict
timeout: int
address: str | None
def __init__(
self,
scheduler_kwargs: dict | None = None,
timeout: int = 30,
):
self.scheduler_kwargs = {
"scheduler_file": os.path.join(tempfile.gettempdir(), str(uuid.uuid4()))
}
if scheduler_kwargs:
self.scheduler_kwargs.update(scheduler_kwargs)
self.timeout = timeout
super().__init__()
async def _start(self):
cmd = [
sys.executable,
"-m",
"dask",
"spec",
"--spec",
json.dumps(
{"cls": "distributed.Scheduler", "opts": {**self.scheduler_kwargs}}
),
]
logger.info(" ".join(cmd))
deadline = Deadline.after(self.timeout)
self.process = await asyncio.create_subprocess_exec(
*cmd,
stderr=asyncio.subprocess.PIPE,
)
scheduler_file = Path(self.scheduler_kwargs["scheduler_file"])
while not (
deadline.expired
or scheduler_file.exists()
or self.process.returncode is not None
):
await asyncio.sleep(0.1)
if deadline.expired or self.process.returncode is not None:
assert self.process.stderr
logger.error((await self.process.stderr.read()).decode())
if deadline.expired:
raise RuntimeError(f"Scheduler failed to start within {self.timeout}s")
raise RuntimeError(
f"Scheduler failed to start and exited with code {self.process.returncode}"
)
with scheduler_file.open(mode="r") as f:
identity = json.load(f)
self.address = identity["address"]
logger.info("Scheduler at %r", self.address)
class SubprocessWorker(Subprocess):
"""A local Dask worker running in a dedicated subprocess
Parameters
----------
scheduler:
Address of the scheduler
worker_class:
Python class to use to create the worker, defaults to 'distributed.Nanny'
name:
Name of the worker
worker_kwargs:
Keywords to pass on to the ``Worker`` class constructor
"""
name: str | None
scheduler: str
worker_class: str
worker_kwargs: dict
def __init__(
self,
scheduler: str,
worker_class: str = "distributed.Nanny",
name: str | None = None,
worker_kwargs: dict | None = None,
) -> None:
self.name = name
self.scheduler = scheduler
self.worker_class = worker_class
self.worker_kwargs = copy.copy(worker_kwargs or {})
super().__init__()
async def _start(self) -> None:
cmd = [
sys.executable,
"-m",
"dask",
"spec",
self.scheduler,
"--spec",
json.dumps({"cls": self.worker_class, "opts": {**self.worker_kwargs}}),
]
logger.info(" ".join(cmd))
self.process = await asyncio.create_subprocess_exec(*cmd)
def SubprocessCluster(
host: str | None = None,
scheduler_port: int = 0,
scheduler_kwargs: dict | None = None,
dashboard_address: str = ":8787",
worker_class: str = "distributed.Nanny",
n_workers: int | None = None,
threads_per_worker: int | None = None,
worker_kwargs: dict | None = None,
silence_logs: int = logging.WARN,
**kwargs: Any,
) -> SpecCluster:
"""Create a scheduler and workers that run in dedicated subprocesses
This creates a "cluster" of a scheduler and workers running in dedicated subprocesses.
.. warning::
This function is experimental
Parameters
----------
host:
Host address on which the scheduler will listen, defaults to localhost
scheduler_port:
Port fo the scheduler, defaults to 0 to choose a random port
scheduler_kwargs:
Keywords to pass on to scheduler
dashboard_address:
Address on which to listen for the Bokeh diagnostics server like
'localhost:8787' or '0.0.0.0:8787', defaults to ':8787'
Set to ``None`` to disable the dashboard.
Use ':0' for a random port.
worker_class:
Worker class to instantiate workers from, defaults to 'distributed.Nanny'
n_workers:
Number of workers to start
threads:
Number of threads per each worker
worker_kwargs:
Keywords to pass on to the ``Worker`` class constructor
silence_logs:
Level of logs to print out to stdout, defaults to ``logging.WARN``
Use a falsy value like False or None to disable log silencing.
Examples
--------
>>> cluster = SubprocessCluster() # Create a subprocess cluster #doctest: +SKIP
>>> cluster # doctest: +SKIP
SubprocessCluster(SubprocessCluster, 'tcp://127.0.0.1:61207', workers=5, threads=10, memory=16.00 GiB)
>>> c = Client(cluster) # connect to subprocess cluster # doctest: +SKIP
Scale the cluster to three workers
>>> cluster.scale(3) # doctest: +SKIP
"""
if WINDOWS:
# FIXME: distributed#7434
raise RuntimeError("SubprocessCluster does not support Windows.")
if not host:
host = "127.0.0.1"
worker_kwargs = worker_kwargs or {}
scheduler_kwargs = scheduler_kwargs or {}
if n_workers is None and threads_per_worker is None:
n_workers, threads_per_worker = nprocesses_nthreads()
if n_workers is None and threads_per_worker is not None:
n_workers = max(1, CPU_COUNT // threads_per_worker)
if n_workers and threads_per_worker is None:
# Overcommit threads per worker, rather than undercommit
threads_per_worker = max(1, int(math.ceil(CPU_COUNT / n_workers)))
if n_workers and "memory_limit" not in worker_kwargs:
worker_kwargs["memory_limit"] = parse_memory_limit(
"auto", 1, n_workers, logger=logger
)
assert n_workers is not None
scheduler_kwargs = toolz.merge(
{
"host": host,
"port": scheduler_port,
"dashboard": dashboard_address is not None,
"dashboard_address": dashboard_address,
},
scheduler_kwargs,
)
worker_kwargs = toolz.merge(
{
"host": host,
"nthreads": threads_per_worker,
"silence_logs": silence_logs,
},
worker_kwargs,
)
scheduler = {
"cls": SubprocessScheduler,
"options": {
"scheduler_kwargs": scheduler_kwargs,
},
}
worker = {
"cls": SubprocessWorker,
"options": {"worker_class": worker_class, "worker_kwargs": worker_kwargs},
}
workers = {i: worker for i in range(n_workers)}
return SpecCluster(
workers=workers,
scheduler=scheduler,
worker=worker,
name="SubprocessCluster",
silence_logs=silence_logs,
**kwargs,
)
|