|
19 | 19 | from distributed.core import Status
|
20 | 20 | from distributed.metrics import time
|
21 | 21 | from distributed.system import MEMORY_LIMIT
|
22 |
| -from distributed.utils import TimeoutError, open_port, sync |
| 22 | +from distributed.utils import Deadline, TimeoutError, open_port, sync |
23 | 23 | from distributed.utils_test import (
|
24 | 24 | assert_can_connect_from_everywhere_4,
|
25 | 25 | assert_can_connect_from_everywhere_4_6,
|
@@ -1255,6 +1255,59 @@ def setup(self, worker=None):
|
1255 | 1255 | import my_nonexistent_library # noqa
|
1256 | 1256 |
|
1257 | 1257 |
|
| 1258 | +class SlowPlugin: |
| 1259 | + def __init__(self, delay=0.1): |
| 1260 | + self.delay = delay |
| 1261 | + |
| 1262 | + def setup(self, worker=None): |
| 1263 | + sleep(self.delay) |
| 1264 | + |
| 1265 | + |
| 1266 | +@pytest.mark.slow() |
| 1267 | +def test_localcluster_plan_requested_observed(): |
| 1268 | + with LocalCluster( |
| 1269 | + n_workers=0, |
| 1270 | + threads_per_worker=1, |
| 1271 | + processes=True, |
| 1272 | + # FIXME: Ideally this would work with an IPC Event or a file to |
| 1273 | + # synchronize instead of sleeping |
| 1274 | + plugins={SlowPlugin(delay=2)}, |
| 1275 | + dashboard_address=":0", |
| 1276 | + ) as cluster: |
| 1277 | + assert len(cluster.plan) == 0 |
| 1278 | + assert len(cluster.requested) == 0 |
| 1279 | + assert len(cluster.observed) == 0 |
| 1280 | + |
| 1281 | + cluster.scale(1) |
| 1282 | + assert len(cluster.plan) == 1 |
| 1283 | + assert len(cluster.requested) == 0 |
| 1284 | + assert len(cluster.observed) == 0 |
| 1285 | + |
| 1286 | + # This should pretty much trigger once we had the chance to run an event |
| 1287 | + # loop tick |
| 1288 | + dl = Deadline.after(1) |
| 1289 | + while not cluster.requested and dl.remaining: |
| 1290 | + sleep(0.01) |
| 1291 | + |
| 1292 | + # The worker is requested. For the LocalCluster this means that the |
| 1293 | + # process is up but for generic SpecCluster implementation this merely |
| 1294 | + # means that an additional worker has been asked for but it is not yet |
| 1295 | + # up and running |
| 1296 | + assert not cluster.scheduler_info["workers"] |
| 1297 | + assert len(cluster.plan) == 1 |
| 1298 | + assert len(cluster.requested) == 1 |
| 1299 | + assert len(cluster.observed) == 0 |
| 1300 | + |
| 1301 | + with Client(cluster) as client: |
| 1302 | + client.wait_for_workers(1) |
| 1303 | + |
| 1304 | + # The worker is fully functional and registered to the scheduler |
| 1305 | + assert cluster.scheduler_info["workers"] |
| 1306 | + assert len(cluster.requested) == 1 |
| 1307 | + assert len(cluster.plan) == 1 |
| 1308 | + assert len(cluster.observed) == 1 |
| 1309 | + |
| 1310 | + |
1258 | 1311 | @pytest.mark.slow
|
1259 | 1312 | def test_localcluster_start_exception(loop):
|
1260 | 1313 | with raises_with_cause(
|
|
0 commit comments