|
| 1 | +import asyncio |
| 2 | +import sys |
| 3 | +import os |
| 4 | +import signal |
| 5 | +from contextlib import suppress |
| 6 | +from enum import Enum |
| 7 | +from typing import Dict, Optional |
| 8 | +import warnings |
| 9 | +from tornado.ioloop import IOLoop |
| 10 | + |
| 11 | +from distributed.core import CommClosedError, Status, rpc |
| 12 | +from distributed.scheduler import Scheduler |
| 13 | +from distributed.utils import LoopRunner, import_term, SyncMethodMixin |
| 14 | +from distributed.worker import Worker |
| 15 | + |
| 16 | + |
| 17 | +# Close gracefully when receiving a SIGINT |
| 18 | +signal.signal(signal.SIGINT, lambda *_: sys.exit()) |
| 19 | + |
| 20 | + |
| 21 | +class Role(Enum): |
| 22 | + """ |
| 23 | + This Enum contains the various roles processes can be. |
| 24 | + """ |
| 25 | + |
| 26 | + worker = "worker" |
| 27 | + scheduler = "scheduler" |
| 28 | + client = "client" |
| 29 | + |
| 30 | + |
| 31 | +class BaseRunner(SyncMethodMixin): |
| 32 | + """Superclass for runner objects. |
| 33 | +
|
| 34 | + This class contains common functionality for Dask cluster runner classes. |
| 35 | +
|
| 36 | + To implement this class, you must provide |
| 37 | +
|
| 38 | + 1. A ``get_role`` method which returns a role from the ``Role`` enum. |
| 39 | + 2. Optionally, a ``set_scheduler_address`` method for the scheduler process to communicate its address. |
| 40 | + 3. A ``get_scheduler_address`` method for all other processed to recieve the scheduler address. |
| 41 | + 4. Optionally, a ``get_worker_name`` to provide a platform specific name to the workers. |
| 42 | + 5. Optionally, a ``before_scheduler_start`` to perform any actions before the scheduler is created. |
| 43 | + 6. Optionally, a ``before_worker_start`` to perform any actions before the worker is created. |
| 44 | + 7. Optionally, a ``before_client_start`` to perform any actions before the client code continues. |
| 45 | + 8. Optionally, a ``on_scheduler_start`` to perform anything on the scheduler once it has started. |
| 46 | + 9. Optionally, a ``on_worker_start`` to perform anything on the worker once it has started. |
| 47 | +
|
| 48 | + For that, you should get the following: |
| 49 | +
|
| 50 | + A context manager and object which can be used within a script that is run in parallel to decide which processes |
| 51 | + run the scheduler, workers and client code. |
| 52 | +
|
| 53 | + """ |
| 54 | + |
| 55 | + __loop: Optional[IOLoop] = None |
| 56 | + |
| 57 | + def __init__( |
| 58 | + self, |
| 59 | + scheduler: bool = True, |
| 60 | + scheduler_options: Dict = None, |
| 61 | + worker_class: str = None, |
| 62 | + worker_options: Dict = None, |
| 63 | + client: bool = True, |
| 64 | + asynchronous: bool = False, |
| 65 | + loop: asyncio.BaseEventLoop = None, |
| 66 | + ): |
| 67 | + self.status = Status.created |
| 68 | + self.scheduler = scheduler |
| 69 | + self.scheduler_address = None |
| 70 | + self.scheduler_comm = None |
| 71 | + self.client = client |
| 72 | + if self.client and not self.scheduler: |
| 73 | + raise RuntimeError("Cannot run client code without a scheduler.") |
| 74 | + self.scheduler_options = ( |
| 75 | + scheduler_options if scheduler_options is not None else {} |
| 76 | + ) |
| 77 | + self.worker_class = ( |
| 78 | + Worker if worker_class is None else import_term(worker_class) |
| 79 | + ) |
| 80 | + self.worker_options = worker_options if worker_options is not None else {} |
| 81 | + self.role = None |
| 82 | + self.__asynchronous = asynchronous |
| 83 | + self._loop_runner = LoopRunner(loop=loop, asynchronous=asynchronous) |
| 84 | + |
| 85 | + if not self.__asynchronous: |
| 86 | + self._loop_runner.start() |
| 87 | + self.sync(self._start) |
| 88 | + |
| 89 | + async def get_role(self) -> str: |
| 90 | + raise NotImplementedError() |
| 91 | + |
| 92 | + async def set_scheduler_address(self, scheduler: Scheduler) -> None: |
| 93 | + raise None |
| 94 | + |
| 95 | + async def get_scheduler_address(self) -> str: |
| 96 | + raise NotImplementedError() |
| 97 | + |
| 98 | + async def get_worker_name(self) -> str: |
| 99 | + return None |
| 100 | + |
| 101 | + async def before_scheduler_start(self) -> None: |
| 102 | + return None |
| 103 | + |
| 104 | + async def before_worker_start(self) -> None: |
| 105 | + return None |
| 106 | + |
| 107 | + async def before_client_start(self) -> None: |
| 108 | + return None |
| 109 | + |
| 110 | + async def on_scheduler_start(self, scheduler: Scheduler) -> None: |
| 111 | + return None |
| 112 | + |
| 113 | + async def on_worker_start(self, worker: Worker) -> None: |
| 114 | + return None |
| 115 | + |
| 116 | + @property |
| 117 | + def loop(self) -> Optional[IOLoop]: |
| 118 | + loop = self.__loop |
| 119 | + if loop is None: |
| 120 | + # If the loop is not running when this is called, the LoopRunner.loop |
| 121 | + # property will raise a DeprecationWarning |
| 122 | + # However subsequent calls might occur - eg atexit, where a stopped |
| 123 | + # loop is still acceptable - so we cache access to the loop. |
| 124 | + self.__loop = loop = self._loop_runner.loop |
| 125 | + return loop |
| 126 | + |
| 127 | + @loop.setter |
| 128 | + def loop(self, value: IOLoop) -> None: |
| 129 | + warnings.warn( |
| 130 | + "setting the loop property is deprecated", DeprecationWarning, stacklevel=2 |
| 131 | + ) |
| 132 | + if value is None: |
| 133 | + raise ValueError("expected an IOLoop, got None") |
| 134 | + self.__loop = value |
| 135 | + |
| 136 | + def __await__(self): |
| 137 | + async def _await(): |
| 138 | + if self.status != Status.running: |
| 139 | + await self._start() |
| 140 | + return self |
| 141 | + |
| 142 | + return _await().__await__() |
| 143 | + |
| 144 | + async def __aenter__(self): |
| 145 | + await self |
| 146 | + return self |
| 147 | + |
| 148 | + async def __aexit__(self, *args): |
| 149 | + await self._close() |
| 150 | + |
| 151 | + def __enter__(self): |
| 152 | + return self.sync(self.__aenter__) |
| 153 | + |
| 154 | + def __exit__(self, typ, value, traceback): |
| 155 | + return self.sync(self.__aexit__) |
| 156 | + |
| 157 | + def __del__(self): |
| 158 | + with suppress(AttributeError, RuntimeError): # during closing |
| 159 | + self.loop.add_callback(self.close) |
| 160 | + |
| 161 | + async def _start(self) -> None: |
| 162 | + self.role = await self.get_role() |
| 163 | + if self.role == Role.scheduler: |
| 164 | + await self.start_scheduler() |
| 165 | + os.kill( |
| 166 | + os.getpid(), signal.SIGINT |
| 167 | + ) # Shutdown with a signal to give the event loop time to close |
| 168 | + elif self.role == Role.worker: |
| 169 | + await self.start_worker() |
| 170 | + os.kill( |
| 171 | + os.getpid(), signal.SIGINT |
| 172 | + ) # Shutdown with a signal to give the event loop time to close |
| 173 | + elif self.role == Role.client: |
| 174 | + self.scheduler_address = await self.get_scheduler_address() |
| 175 | + if self.scheduler_address: |
| 176 | + self.scheduler_comm = rpc(self.scheduler_address) |
| 177 | + await self.before_client_start() |
| 178 | + self.status = Status.running |
| 179 | + |
| 180 | + async def start_scheduler(self) -> None: |
| 181 | + await self.before_scheduler_start() |
| 182 | + async with Scheduler(**self.scheduler_options) as scheduler: |
| 183 | + await self.set_scheduler_address(scheduler) |
| 184 | + await self.on_scheduler_start(scheduler) |
| 185 | + await scheduler.finished() |
| 186 | + |
| 187 | + async def start_worker(self) -> None: |
| 188 | + if ( |
| 189 | + "scheduler_file" not in self.worker_options |
| 190 | + and "scheduler_ip" not in self.worker_options |
| 191 | + ): |
| 192 | + self.worker_options["scheduler_ip"] = await self.get_scheduler_address() |
| 193 | + worker_name = await self.get_worker_name() |
| 194 | + await self.before_worker_start() |
| 195 | + async with self.worker_class(name=worker_name, **self.worker_options) as worker: |
| 196 | + await self.on_worker_start(worker) |
| 197 | + await worker.finished() |
| 198 | + |
| 199 | + async def _close(self) -> None: |
| 200 | + if self.status == Status.running: |
| 201 | + if self.scheduler_comm: |
| 202 | + with suppress(CommClosedError): |
| 203 | + await self.scheduler_comm.terminate() |
| 204 | + self.status = Status.closed |
| 205 | + |
| 206 | + def close(self) -> None: |
| 207 | + return self.sync(self._close) |
| 208 | + |
| 209 | + |
| 210 | +class AsyncCommWorld: |
| 211 | + def __init__(self): |
| 212 | + self.roles = {"scheduler": None, "client": None} |
| 213 | + self.role_lock = asyncio.Lock() |
| 214 | + self.scheduler_address = None |
| 215 | + |
| 216 | + |
| 217 | +class AsyncRunner(BaseRunner): |
| 218 | + def __init__(self, commworld: AsyncCommWorld, *args, **kwargs): |
| 219 | + self.commworld = commworld |
| 220 | + super().__init__(*args, **kwargs) |
| 221 | + |
| 222 | + async def get_role(self) -> str: |
| 223 | + async with self.commworld.role_lock: |
| 224 | + if self.commworld.roles["scheduler"] is None and self.scheduler: |
| 225 | + self.commworld.roles["scheduler"] = self |
| 226 | + return Role.scheduler |
| 227 | + elif self.commworld.roles["client"] is None and self.client: |
| 228 | + self.commworld.roles["client"] = self |
| 229 | + return Role.client |
| 230 | + else: |
| 231 | + return Role.worker |
| 232 | + |
| 233 | + async def set_scheduler_address(self, scheduler: Scheduler) -> None: |
| 234 | + self.commworld.scheduler_address = scheduler.address |
| 235 | + |
| 236 | + async def get_scheduler_address(self) -> str: |
| 237 | + while self.commworld.scheduler_address is None: |
| 238 | + await asyncio.sleep(0.1) |
| 239 | + return self.commworld.scheduler_address |
0 commit comments