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 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739
|
from __future__ import annotations
import asyncio
import logging
import math
from time import sleep
import pytest
import dask
from distributed import (
Adaptive,
Client,
LocalCluster,
Scheduler,
SpecCluster,
Worker,
wait,
)
from distributed.core import Status
from distributed.deploy.cluster import Cluster
from distributed.metrics import time
from distributed.utils_test import (
async_poll_for,
captured_logger,
gen_cluster,
gen_test,
slowinc,
)
def test_adaptive_local_cluster(loop):
with LocalCluster(
n_workers=0,
silence_logs=False,
dashboard_address=":0",
loop=loop,
) as cluster:
alc = cluster.adapt(interval="100 ms")
with Client(cluster, loop=loop) as c:
assert not cluster.scheduler.workers
future = c.submit(lambda x: x + 1, 1)
assert future.result() == 2
assert cluster.scheduler.workers
sleep(0.1)
assert cluster.scheduler.workers
del future
start = time()
while cluster.scheduler.workers:
sleep(0.01)
assert time() < start + 30
assert not cluster.scheduler.workers
@gen_test()
async def test_adaptive_local_cluster_multi_workers():
async with LocalCluster(
n_workers=0,
silence_logs=False,
processes=False,
dashboard_address=":0",
asynchronous=True,
) as cluster:
cluster.scheduler.allowed_failures = 1000
adapt = cluster.adapt(interval="100 ms")
async with Client(cluster, asynchronous=True) as c:
futures = c.map(slowinc, range(100), delay=0.01)
while not cluster.scheduler.workers:
await asyncio.sleep(0.01)
await c.gather(futures)
del futures
while cluster.scheduler.workers:
await asyncio.sleep(0.01)
# no workers for a while
for _ in range(10):
assert not cluster.scheduler.workers
await asyncio.sleep(0.05)
futures = c.map(slowinc, range(100), delay=0.01)
await c.gather(futures)
@gen_test()
async def test_min_max():
async with LocalCluster(
n_workers=0,
silence_logs=False,
processes=False,
dashboard_address=":0",
asynchronous=True,
threads_per_worker=1,
) as cluster:
adapt = cluster.adapt(minimum=1, maximum=2, interval="20 ms", wait_count=10)
async with Client(cluster, asynchronous=True) as c:
start = time()
while not cluster.scheduler.workers:
await asyncio.sleep(0.01)
assert time() < start + 1
await asyncio.sleep(0.2)
assert len(cluster.scheduler.workers) == 1
assert len(adapt.log) == 1 and adapt.log[-1][1] == {"status": "up", "n": 1}
futures = c.map(slowinc, range(100), delay=0.1)
start = time()
while len(cluster.scheduler.workers) < 2:
await asyncio.sleep(0.01)
assert time() < start + 1
assert len(cluster.scheduler.workers) == 2
await asyncio.sleep(0.5)
assert len(cluster.scheduler.workers) == 2
assert len(cluster.workers) == 2
assert len(adapt.log) == 2 and all(
d["status"] == "up" for _, d in adapt.log
)
del futures
start = time()
while len(cluster.scheduler.workers) != 1:
await asyncio.sleep(0.01)
assert time() < start + 2
assert adapt.log[-1][1]["status"] == "down"
@gen_test()
async def test_avoid_churn():
"""We want to avoid creating and deleting workers frequently
Instead we want to wait a few beats before removing a worker in case the
user is taking a brief pause between work
"""
async with LocalCluster(
n_workers=0,
asynchronous=True,
processes=False,
silence_logs=False,
dashboard_address=":0",
) as cluster:
async with Client(cluster, asynchronous=True) as client:
adapt = cluster.adapt(interval="20 ms", wait_count=5)
for i in range(10):
await client.submit(slowinc, i, delay=0.040)
await asyncio.sleep(0.040)
assert len(adapt.log) == 1
@gen_test()
async def test_adapt_quickly():
"""We want to avoid creating and deleting workers frequently
Instead we want to wait a few beats before removing a worker in case the
user is taking a brief pause between work
"""
async with (
LocalCluster(
n_workers=0,
asynchronous=True,
processes=False,
silence_logs=False,
dashboard_address=":0",
threads_per_worker=1,
) as cluster,
Client(cluster, asynchronous=True) as client,
):
adapt = cluster.adapt(interval="20 ms", wait_count=5, maximum=10)
future = client.submit(slowinc, 1, delay=0.100)
await wait(future)
assert len(adapt.log) == 1
# Scale up when there is plenty of available work
futures = client.map(slowinc, range(2, 1002), delay=0.100)
while len(adapt.log) == 1:
await asyncio.sleep(0.01)
assert len(adapt.log) == 2
assert adapt.log[-1][1]["status"] == "up"
d = [x for x in adapt.log[-1] if isinstance(x, dict)][0]
assert 2 < d["n"] <= adapt.maximum
while len(cluster.workers) < adapt.maximum:
await asyncio.sleep(0.01)
del futures
while len(cluster.scheduler.tasks) > 1:
await asyncio.sleep(0.01)
await cluster
while (
len(cluster.scheduler.workers) > 1
or len(cluster.worker_spec) > 1
or len(cluster.workers) > 1
):
await asyncio.sleep(0.01)
# Don't scale up for large sequential computations
x = await client.scatter(1)
for _ in range(100):
x = client.submit(slowinc, x)
await asyncio.sleep(0.1)
assert len(cluster.workers) == 1
@gen_test()
async def test_adapt_down():
"""Ensure that redefining adapt with a lower maximum removes workers"""
async with (
LocalCluster(
n_workers=0,
asynchronous=True,
processes=False,
silence_logs=False,
dashboard_address=":0",
) as cluster,
Client(cluster, asynchronous=True) as client,
):
cluster.adapt(interval="20ms", maximum=5)
futures = client.map(slowinc, range(1000), delay=0.1)
while len(cluster.scheduler.workers) < 5:
await asyncio.sleep(0.1)
cluster.adapt(maximum=2)
start = time()
while len(cluster.scheduler.workers) != 2:
await asyncio.sleep(0.1)
assert time() < start + 60
@gen_test()
async def test_no_more_workers_than_tasks():
with dask.config.set(
{"distributed.scheduler.default-task-durations": {"slowinc": 1000}}
):
async with LocalCluster(
n_workers=0,
silence_logs=False,
processes=False,
dashboard_address=":0",
asynchronous=True,
) as cluster:
adapt = cluster.adapt(minimum=0, maximum=4, interval="10 ms")
async with Client(cluster, asynchronous=True) as client:
await client.submit(slowinc, 1, delay=0.100)
assert len(cluster.scheduler.workers) <= 1
def test_basic_no_loop(cleanup):
loop = None
try:
with LocalCluster(
n_workers=0, silence_logs=False, dashboard_address=":0", loop=None
) as cluster:
with Client(cluster) as client:
cluster.adapt()
future = client.submit(lambda x: x + 1, 1)
assert future.result() == 2
loop = cluster.loop
finally:
assert loop is None or not loop.asyncio_loop.is_running()
@pytest.mark.parametrize("target_duration", [5, 1])
def test_target_duration(target_duration):
@gen_test()
async def _test():
with dask.config.set(
{
"distributed.scheduler.default-task-durations": {"slowinc": 1},
# adaptive target for queued tasks doesn't yet consider default or learned task durations
"distributed.scheduler.worker-saturation": float("inf"),
}
):
async with LocalCluster(
n_workers=0,
asynchronous=True,
processes=False,
silence_logs=False,
dashboard_address=":0",
) as cluster:
adapt = cluster.adapt(
interval="20ms", minimum=2, target_duration=target_duration
)
# FIXME: LocalCluster is starting workers with CPU_COUNT threads
# each
# The default target duration is set to 1s
max_scaleup = 5
n_tasks = target_duration * dask.system.CPU_COUNT * max_scaleup
async with Client(cluster, asynchronous=True) as client:
await client.wait_for_workers(2)
futures = client.map(slowinc, range(n_tasks), delay=0.3)
await wait(futures)
scaleup_recs = [
msg[1]["n"] for msg in adapt.log if msg[1].get("status") == "up"
]
assert 2 <= min(scaleup_recs) < max(scaleup_recs) <= max_scaleup
_test()
@gen_test()
async def test_worker_keys():
"""Ensure that redefining adapt with a lower maximum removes workers"""
async with SpecCluster(
scheduler={"cls": Scheduler, "options": {"dashboard_address": ":0"}},
workers={
"a-1": {"cls": Worker},
"a-2": {"cls": Worker},
"b-1": {"cls": Worker},
"b-2": {"cls": Worker},
},
asynchronous=True,
) as cluster:
def key(ws):
return ws.name.split("-")[0]
cluster._adaptive_options = {"worker_key": key}
adaptive = cluster.adapt(minimum=1)
await adaptive.adapt()
while len(cluster.scheduler.workers) == 4:
await asyncio.sleep(0.01)
names = {ws.name for ws in cluster.scheduler.workers.values()}
assert names == {"a-1", "a-2"} or names == {"b-1", "b-2"}
@gen_test()
async def test_adapt_cores_memory():
async with LocalCluster(
n_workers=0,
threads_per_worker=2,
memory_limit="3 GB",
silence_logs=False,
processes=False,
dashboard_address=":0",
asynchronous=True,
) as cluster:
adapt = cluster.adapt(minimum_cores=3, maximum_cores=9)
assert adapt.minimum == 2
assert adapt.maximum == 4
adapt = cluster.adapt(minimum_memory="7GB", maximum_memory="20 GB")
assert adapt.minimum == 3
assert adapt.maximum == 6
adapt = cluster.adapt(
minimum_cores=1,
minimum_memory="7GB",
maximum_cores=10,
maximum_memory="1 TB",
)
assert adapt.minimum == 3
assert adapt.maximum == 5
@gen_test()
async def test_adaptive_config():
async with LocalCluster(
n_workers=0,
asynchronous=True,
silence_logs=False,
dashboard_address=":0",
) as cluster:
with dask.config.set(
{"distributed.adaptive.minimum": 10, "distributed.adaptive.wait-count": 8}
):
try:
adapt = Adaptive(cluster, interval="5s")
assert adapt.minimum == 10
assert adapt.maximum == math.inf
assert adapt.interval == 5
assert adapt.wait_count == 8
finally:
adapt.stop()
@gen_test()
async def test_update_adaptive():
async with LocalCluster(
n_workers=0,
threads_per_worker=2,
memory_limit="3 GB",
silence_logs=False,
processes=False,
dashboard_address=":0",
asynchronous=True,
) as cluster:
first = cluster.adapt(maximum=1)
second = cluster.adapt(maximum=2)
await asyncio.sleep(0.2)
assert first.state == "stopped"
assert second.state == "running"
assert first.periodic_callback is None
assert second.periodic_callback.is_running()
@gen_test()
async def test_adaptive_no_memory_limit():
"""Test that adapt() does not keep creating workers when no memory limit is set"""
async with LocalCluster(
n_workers=0,
threads_per_worker=1,
memory_limit=0,
asynchronous=True,
dashboard_address=":0",
) as cluster:
cluster.adapt(minimum=1, maximum=10, interval="1 ms")
async with Client(cluster, asynchronous=True) as client:
await client.gather(client.map(slowinc, range(5), delay=0.35))
assert (
sum(
state[1]["n"]
for state in cluster._adaptive.log
if state[1]["status"] == "up"
)
<= 5
)
@gen_test()
async def test_adapt_gets_stopped_on_cluster_close():
class MyCluster(Cluster):
pass
async with MyCluster(asynchronous=True) as cluster:
adapt = cluster.adapt(minimum=1, maximum=10, interval="10ms")
while adapt.state != "running":
await asyncio.sleep(0.01)
await cluster.close()
assert adapt.state == "stopped"
@gen_test()
async def test_scale_needs_to_be_awaited():
"""
This tests that the adaptive class works fine if the scale method uses the
`sync` method to schedule its task instead of loop.add_callback
"""
class RequiresAwaitCluster(LocalCluster):
def scale(self, n):
# super invocation in the nested function scope is messy
method = super().scale
async def _():
return method(n)
return self.sync(_)
async with RequiresAwaitCluster(
n_workers=0, asynchronous=True, dashboard_address=":0"
) as cluster:
async with Client(cluster, asynchronous=True) as client:
futures = client.map(slowinc, range(5), delay=0.05)
assert len(cluster.workers) == 0
cluster.adapt()
await client.gather(futures)
del futures
await async_poll_for(lambda: not cluster.workers, 10)
@gen_test()
async def test_adaptive_stopped():
"""
We should ensure that the adapt PC is actually stopped once the cluster
stops.
"""
async with LocalCluster(
n_workers=0, asynchronous=True, dashboard_address=":0"
) as cluster:
instance = cluster.adapt(interval="10ms")
await async_poll_for(lambda: instance.state == "running", timeout=5)
assert instance.periodic_callback is not None
assert instance.periodic_callback.is_running()
pc = instance.periodic_callback
await async_poll_for(lambda: instance.state == "stopped", timeout=5)
assert not pc.is_running()
@pytest.mark.parametrize("saturation", [1, float("inf")])
@gen_cluster(
client=True,
nthreads=[],
config={
"distributed.scheduler.default-task-durations": {"slowinc": 1000},
},
)
async def test_scale_up_large_tasks(c, s, saturation):
s.WORKER_SATURATION = saturation
futures = c.map(slowinc, range(10))
while not s.tasks:
await asyncio.sleep(0.001)
assert s.adaptive_target() == 10
more_futures = c.map(slowinc, range(200))
while len(s.tasks) != 200:
await asyncio.sleep(0.001)
assert s.adaptive_target() == 200
@gen_cluster(
client=True,
nthreads=[("", 5)],
config={"distributed.scheduler.default-task-durations": {"slowinc": 1000}},
)
async def test_respect_average_nthreads(c, s, w):
futures = c.map(slowinc, range(10))
while not s.tasks:
await asyncio.sleep(0.001)
assert s.adaptive_target() == 2
more_futures = c.map(slowinc, range(200))
while len(s.tasks) != 200:
await asyncio.sleep(0.001)
assert s.adaptive_target() == 40
class MyAdaptive(Adaptive):
def __init__(self, *args, interval=None, **kwargs):
super().__init__(*args, interval=interval, **kwargs)
self._target = 0
self._log = []
self._observed = set()
self._plan = set()
self._requested = set()
@property
def observed(self):
return self._observed
@property
def plan(self):
return self._plan
@property
def requested(self):
return self._requested
async def target(self):
return self._target
async def scale_up(self, n=0):
self._plan = self._requested = set(range(n))
async def scale_down(self, workers=()):
for collection in [self.plan, self.requested, self.observed]:
for w in workers:
collection.discard(w)
@gen_test()
async def test_adaptive_stops_on_cluster_status_change():
async with LocalCluster(
n_workers=0,
asynchronous=True,
silence_logs=False,
dashboard_address=":0",
) as cluster:
adapt = Adaptive(cluster, interval="100 ms")
assert adapt.state == "starting"
await async_poll_for(lambda: adapt.state == "running", timeout=5)
assert adapt.periodic_callback
assert adapt.periodic_callback.is_running()
try:
cluster.status = Status.closing
await async_poll_for(lambda: adapt.state != "running", timeout=5)
assert adapt.state == "stopped"
assert not adapt.periodic_callback
finally:
# Set back to running to let normal shutdown do its thing
cluster.status = Status.running
@gen_test()
async def test_interval():
async with LocalCluster(
n_workers=0,
asynchronous=True,
silence_logs=False,
dashboard_address=":0",
) as cluster:
adapt = MyAdaptive(cluster=cluster, interval="100 ms")
assert not adapt.plan
for i in [0, 3, 1]:
start = time()
adapt._target = i
while len(adapt.plan) != i:
await asyncio.sleep(0.01)
assert time() < start + 2
adapt.stop()
await asyncio.sleep(0.05)
adapt._target = 10
await asyncio.sleep(0.02)
assert len(adapt.plan) == 1 # last value from before, unchanged
@gen_test()
async def test_adapt_logs_error_in_safe_target():
class BadAdaptive(MyAdaptive):
"""Adaptive subclass which raises an OSError when attempting to adapt
We use this to check that error handling works properly
"""
def safe_target(self):
raise OSError()
async with LocalCluster(
n_workers=0,
asynchronous=True,
silence_logs=False,
dashboard_address=":0",
) as cluster:
with captured_logger(
"distributed.deploy.adaptive", level=logging.WARNING
) as log:
adapt = cluster.adapt(
Adaptive=BadAdaptive, minimum=1, maximum=4, interval="10ms"
)
while "encountered an error" not in log.getvalue():
await asyncio.sleep(0.01)
assert "stop" not in log.getvalue()
assert adapt.state == "running"
assert adapt.periodic_callback
assert adapt.periodic_callback.is_running()
@gen_test()
async def test_adapt_callback_logs_error_in_scale_down():
class BadAdaptive(MyAdaptive):
async def scale_down(self, workers=None):
raise OSError()
async with LocalCluster(
n_workers=0,
asynchronous=True,
silence_logs=False,
dashboard_address=":0",
) as cluster:
adapt = cluster.adapt(
Adaptive=BadAdaptive, minimum=1, maximum=4, wait_count=0, interval="10ms"
)
adapt._target = 2
await async_poll_for(lambda: adapt.state == "running", timeout=5)
assert adapt.periodic_callback.is_running()
await adapt.adapt()
assert len(adapt.plan) == 2
assert len(adapt.requested) == 2
with captured_logger(
"distributed.deploy.adaptive", level=logging.WARNING
) as log:
adapt._target = 0
while "encountered an error" not in log.getvalue():
await asyncio.sleep(0.01)
assert "stop" not in log.getvalue()
assert not adapt._adapting
assert adapt.periodic_callback
assert adapt.periodic_callback.is_running()
@pytest.mark.parametrize("wait_until_running", [True, False])
@gen_test()
async def test_adaptive_logs_stopping_once(wait_until_running):
async with LocalCluster(
n_workers=0,
asynchronous=True,
silence_logs=False,
dashboard_address=":0",
) as cluster:
with captured_logger("distributed.deploy.adaptive") as log:
adapt = cluster.adapt(Adaptive=MyAdaptive, interval="100ms")
if wait_until_running:
await async_poll_for(lambda: adapt.state == "running", timeout=5)
assert adapt.periodic_callback
assert adapt.periodic_callback.is_running()
pc = adapt.periodic_callback
else:
assert adapt.periodic_callback
assert not adapt.periodic_callback.is_running()
pc = adapt.periodic_callback
adapt.stop()
adapt.stop()
assert adapt.state == "stopped"
assert not adapt.periodic_callback
assert not pc.is_running()
lines = log.getvalue().splitlines()
assert sum("Adaptive scaling stopped" in line for line in lines) == 1
@gen_test()
async def test_adapt_stop_del():
async with LocalCluster(
n_workers=0,
asynchronous=True,
silence_logs=False,
dashboard_address=":0",
) as cluster:
adapt = cluster.adapt(Adaptive=MyAdaptive, interval="100ms")
pc = adapt.periodic_callback
await async_poll_for(lambda: adapt.state == "running", timeout=5) # noqa: F821
# Remove reference of adaptive object from cluster
cluster._adaptive = None
del adapt
await async_poll_for(lambda: not pc.is_running(), timeout=5)
|