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 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827
|
import asyncio
import itertools
import logging
import random
import sys
import weakref
from operator import mul
from time import sleep
import dask
import pytest
from distributed import Nanny, Worker, wait, worker_client
from distributed.config import config
from distributed.metrics import time
from distributed.scheduler import key_split
from distributed.system import MEMORY_LIMIT
from distributed.utils_test import (
captured_logger,
gen_cluster,
inc,
nodebug_setup_module,
nodebug_teardown_module,
slowadd,
slowidentity,
slowinc,
)
from tlz import concat, sliding_window
# Most tests here are timing-dependent
setup_module = nodebug_setup_module
teardown_module = nodebug_teardown_module
@pytest.mark.skipif(
not sys.platform.startswith("linux"), reason="Need 127.0.0.2 to mean localhost"
)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 2), ("127.0.0.2", 2)], timeout=20)
async def test_work_stealing(c, s, a, b):
[x] = await c._scatter([1], workers=a.address)
futures = c.map(slowadd, range(50), [x] * 50)
await wait(futures)
assert len(a.data) > 10
assert len(b.data) > 10
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
async def test_dont_steal_expensive_data_fast_computation(c, s, a, b):
np = pytest.importorskip("numpy")
x = c.submit(np.arange, 1000000, workers=a.address)
await wait([x])
future = c.submit(np.sum, [1], workers=a.address) # learn that sum is fast
await wait([future])
cheap = [
c.submit(np.sum, x, pure=False, workers=a.address, allow_other_workers=True)
for i in range(10)
]
await wait(cheap)
assert len(s.who_has[x.key]) == 1
assert len(b.data) == 0
assert len(a.data) == 12
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
async def test_steal_cheap_data_slow_computation(c, s, a, b):
x = c.submit(slowinc, 100, delay=0.1) # learn that slowinc is slow
await wait(x)
futures = c.map(
slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True
)
await wait(futures)
assert abs(len(a.data) - len(b.data)) <= 5
@pytest.mark.avoid_travis
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
async def test_steal_expensive_data_slow_computation(c, s, a, b):
np = pytest.importorskip("numpy")
x = c.submit(slowinc, 100, delay=0.2, workers=a.address)
await wait(x) # learn that slowinc is slow
x = c.submit(np.arange, 1000000, workers=a.address) # put expensive data
await wait(x)
slow = [c.submit(slowinc, x, delay=0.1, pure=False) for i in range(20)]
await wait(slow)
assert len(s.who_has[x.key]) > 1
assert b.data # not empty
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 10)
async def test_worksteal_many_thieves(c, s, *workers):
x = c.submit(slowinc, -1, delay=0.1)
await x
xs = c.map(slowinc, [x] * 100, pure=False, delay=0.1)
await wait(xs)
for w, keys in s.has_what.items():
assert 2 < len(keys) < 30
assert len(s.who_has[x.key]) > 1
assert sum(map(len, s.has_what.values())) < 150
@pytest.mark.xfail(reason="GH#3574")
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
async def test_dont_steal_unknown_functions(c, s, a, b):
futures = c.map(inc, range(100), workers=a.address, allow_other_workers=True)
await wait(futures)
assert len(a.data) >= 95, [len(a.data), len(b.data)]
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
async def test_eventually_steal_unknown_functions(c, s, a, b):
futures = c.map(
slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True
)
await wait(futures)
assert len(a.data) >= 3, [len(a.data), len(b.data)]
assert len(b.data) >= 3, [len(a.data), len(b.data)]
@pytest.mark.skip(reason="")
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3)
async def test_steal_related_tasks(e, s, a, b, c):
futures = e.map(
slowinc, range(20), delay=0.05, workers=a.address, allow_other_workers=True
)
await wait(futures)
nearby = 0
for f1, f2 in sliding_window(2, futures):
if s.who_has[f1.key] == s.who_has[f2.key]:
nearby += 1
assert nearby > 10
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 10, timeout=1000)
async def test_dont_steal_fast_tasks_compute_time(c, s, *workers):
np = pytest.importorskip("numpy")
x = c.submit(np.random.random, 10000000, workers=workers[0].address)
def do_nothing(x, y=None):
pass
# execute and measure runtime once
await wait(c.submit(do_nothing, 1))
futures = c.map(do_nothing, range(1000), y=x)
await wait(futures)
assert len(s.who_has[x.key]) == 1
assert len(s.has_what[workers[0].address]) == 1001
@gen_cluster(client=True)
async def test_dont_steal_fast_tasks_blacklist(c, s, a, b):
# create a dependency
x = c.submit(slowinc, 1, workers=[b.address])
# If the blacklist of fast tasks is tracked somewhere else, this needs to be
# changed. This test requires *any* key which is blacklisted.
from distributed.stealing import fast_tasks
blacklisted_key = next(iter(fast_tasks))
def fast_blacklisted(x, y=None):
# The task should observe a certain computation time such that we can
# ensure that it is not stolen due to the blacklisting. If it is too
# fast, the standard mechanism shouldn't allow stealing
import time
time.sleep(0.01)
futures = c.map(
fast_blacklisted,
range(100),
y=x,
# Submit the task to one worker but allow it to be distributed else,
# i.e. this is not a task restriction
workers=[a.address],
allow_other_workers=True,
key=blacklisted_key,
)
await wait(futures)
# The +1 is the dependency we initially submitted to worker B
assert len(s.has_what[a.address]) == 101
assert len(s.has_what[b.address]) == 1
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)], timeout=20)
async def test_new_worker_steals(c, s, a):
await wait(c.submit(slowinc, 1, delay=0.01))
futures = c.map(slowinc, range(100), delay=0.05)
total = c.submit(sum, futures)
while len(a.tasks) < 10:
await asyncio.sleep(0.01)
b = await Worker(s.address, loop=s.loop, nthreads=1, memory_limit=MEMORY_LIMIT)
result = await total
assert result == sum(map(inc, range(100)))
for w in [a, b]:
assert all(isinstance(v, int) for v in w.data.values())
assert b.data
await b.close()
@gen_cluster(client=True, timeout=20)
async def test_work_steal_no_kwargs(c, s, a, b):
await wait(c.submit(slowinc, 1, delay=0.05))
futures = c.map(
slowinc, range(100), workers=a.address, allow_other_workers=True, delay=0.05
)
await wait(futures)
assert 20 < len(a.data) < 80
assert 20 < len(b.data) < 80
total = c.submit(sum, futures)
result = await total
assert result == sum(map(inc, range(100)))
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.1", 2)])
async def test_dont_steal_worker_restrictions(c, s, a, b):
future = c.submit(slowinc, 1, delay=0.10, workers=a.address)
await future
futures = c.map(slowinc, range(100), delay=0.1, workers=a.address)
while len(a.tasks) + len(b.tasks) < 100:
await asyncio.sleep(0.01)
assert len(a.tasks) == 100
assert len(b.tasks) == 0
result = s.extensions["stealing"].balance()
await asyncio.sleep(0.1)
assert len(a.tasks) == 100
assert len(b.tasks) == 0
@gen_cluster(
client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.1", 2), ("127.0.0.1", 2)]
)
async def test_steal_worker_restrictions(c, s, wa, wb, wc):
future = c.submit(slowinc, 1, delay=0.1, workers={wa.address, wb.address})
await future
ntasks = 100
futures = c.map(slowinc, range(ntasks), delay=0.1, workers={wa.address, wb.address})
while sum(len(w.tasks) for w in [wa, wb, wc]) < ntasks:
await asyncio.sleep(0.01)
assert 0 < len(wa.tasks) < ntasks
assert 0 < len(wb.tasks) < ntasks
assert len(wc.tasks) == 0
s.extensions["stealing"].balance()
await asyncio.sleep(0.1)
assert 0 < len(wa.tasks) < ntasks
assert 0 < len(wb.tasks) < ntasks
assert len(wc.tasks) == 0
@pytest.mark.skipif(
not sys.platform.startswith("linux"), reason="Need 127.0.0.2 to mean localhost"
)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.2", 1)])
async def test_dont_steal_host_restrictions(c, s, a, b):
future = c.submit(slowinc, 1, delay=0.10, workers=a.address)
await future
futures = c.map(slowinc, range(100), delay=0.1, workers="127.0.0.1")
while len(a.tasks) + len(b.tasks) < 100:
await asyncio.sleep(0.01)
assert len(a.tasks) == 100
assert len(b.tasks) == 0
result = s.extensions["stealing"].balance()
await asyncio.sleep(0.1)
assert len(a.tasks) == 100
assert len(b.tasks) == 0
@pytest.mark.skipif(
not sys.platform.startswith("linux"), reason="Need 127.0.0.2 to mean localhost"
)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.2", 2)])
async def test_steal_host_restrictions(c, s, wa, wb):
future = c.submit(slowinc, 1, delay=0.10, workers=wa.address)
await future
ntasks = 100
futures = c.map(slowinc, range(ntasks), delay=0.1, workers="127.0.0.1")
while len(wa.tasks) < ntasks:
await asyncio.sleep(0.01)
assert len(wa.tasks) == ntasks
assert len(wb.tasks) == 0
wc = await Worker(s.address, nthreads=1)
start = time()
while not wc.tasks or len(wa.tasks) == ntasks:
await asyncio.sleep(0.01)
assert time() < start + 3
await asyncio.sleep(0.1)
assert 0 < len(wa.tasks) < ntasks
assert len(wb.tasks) == 0
assert 0 < len(wc.tasks) < ntasks
@gen_cluster(
client=True, nthreads=[("127.0.0.1", 1, {"resources": {"A": 2}}), ("127.0.0.1", 1)]
)
async def test_dont_steal_resource_restrictions(c, s, a, b):
future = c.submit(slowinc, 1, delay=0.10, workers=a.address)
await future
futures = c.map(slowinc, range(100), delay=0.1, resources={"A": 1})
while len(a.tasks) + len(b.tasks) < 100:
await asyncio.sleep(0.01)
assert len(a.tasks) == 100
assert len(b.tasks) == 0
result = s.extensions["stealing"].balance()
await asyncio.sleep(0.1)
assert len(a.tasks) == 100
assert len(b.tasks) == 0
@gen_cluster(
client=True, nthreads=[("127.0.0.1", 1, {"resources": {"A": 2}})], timeout=3
)
async def test_steal_resource_restrictions(c, s, a):
future = c.submit(slowinc, 1, delay=0.10, workers=a.address)
await future
futures = c.map(slowinc, range(100), delay=0.2, resources={"A": 1})
while len(a.tasks) < 101:
await asyncio.sleep(0.01)
assert len(a.tasks) == 101
b = await Worker(s.address, loop=s.loop, nthreads=1, resources={"A": 4})
start = time()
while not b.tasks or len(a.tasks) == 101:
await asyncio.sleep(0.01)
assert time() < start + 3
assert len(b.tasks) > 0
assert len(a.tasks) < 101
await b.close()
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 5, timeout=20)
async def test_balance_without_dependencies(c, s, *workers):
s.extensions["stealing"]._pc.callback_time = 20
def slow(x):
y = random.random() * 0.1
sleep(y)
return y
futures = c.map(slow, range(100))
await wait(futures)
durations = [sum(w.data.values()) for w in workers]
assert max(durations) / min(durations) < 3
@gen_cluster(client=True, nthreads=[("127.0.0.1", 4)] * 2)
async def test_dont_steal_executing_tasks(c, s, a, b):
futures = c.map(
slowinc, range(4), delay=0.1, workers=a.address, allow_other_workers=True
)
await wait(futures)
assert len(a.data) == 4
assert len(b.data) == 0
@gen_cluster(
client=True,
nthreads=[("127.0.0.1", 1)] * 10,
config={"distributed.scheduler.default-task-durations": {"slowidentity": 0.2}},
)
async def test_dont_steal_few_saturated_tasks_many_workers(c, s, a, *rest):
s.extensions["stealing"]._pc.callback_time = 20
x = c.submit(mul, b"0", 100000000, workers=a.address) # 100 MB
await wait(x)
futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(2)]
await wait(futures)
assert len(a.data) == 3
assert not any(w.tasks for w in rest)
@gen_cluster(
client=True,
nthreads=[("127.0.0.1", 1)] * 10,
worker_kwargs={"memory_limit": MEMORY_LIMIT},
config={"distributed.scheduler.default-task-durations": {"slowidentity": 0.2}},
)
async def test_steal_when_more_tasks(c, s, a, *rest):
s.extensions["stealing"]._pc.callback_time = 20
x = c.submit(mul, b"0", 50000000, workers=a.address) # 50 MB
await wait(x)
futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(20)]
start = time()
while not any(w.tasks for w in rest):
await asyncio.sleep(0.01)
assert time() < start + 1
@gen_cluster(
client=True,
nthreads=[("127.0.0.1", 1)] * 10,
config={
"distributed.scheduler.default-task-durations": {
"slowidentity": 0.2,
"slow2": 1,
}
},
)
async def test_steal_more_attractive_tasks(c, s, a, *rest):
def slow2(x):
sleep(1)
return x
s.extensions["stealing"]._pc.callback_time = 20
x = c.submit(mul, b"0", 100000000, workers=a.address) # 100 MB
await wait(x)
futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(10)]
future = c.submit(slow2, x, priority=-1)
while not any(w.tasks for w in rest):
await asyncio.sleep(0.01)
# good future moves first
assert any(future.key in w.tasks for w in rest)
def func(x):
sleep(1)
async def assert_balanced(inp, expected, c, s, *workers):
steal = s.extensions["stealing"]
steal._pc.stop()
counter = itertools.count()
tasks = list(concat(inp))
data_seq = itertools.count()
futures = []
for w, ts in zip(workers, inp):
for t in sorted(ts, reverse=True):
if t:
[dat] = await c.scatter([next(data_seq)], workers=w.address)
ts = s.tasks[dat.key]
# Ensure scheduler state stays consistent
old_nbytes = ts.nbytes
ts.nbytes = int(s.bandwidth * t)
for ws in ts.who_has:
ws.nbytes += ts.nbytes - old_nbytes
else:
dat = 123
i = next(counter)
f = c.submit(
func,
dat,
key="%d-%d" % (int(t), i),
workers=w.address,
allow_other_workers=True,
pure=False,
priority=-i,
)
futures.append(f)
while len(s.rprocessing) < len(futures):
await asyncio.sleep(0.001)
for i in range(10):
steal.balance()
while steal.in_flight:
await asyncio.sleep(0.001)
result = [
sorted([int(key_split(k)) for k in s.processing[w.address]], reverse=True)
for w in workers
]
result2 = sorted(result, reverse=True)
expected2 = sorted(expected, reverse=True)
if config.get("pdb-on-err"):
if result2 != expected2:
import pdb
pdb.set_trace()
if result2 == expected2:
return
raise Exception("Expected: {}; got: {}".format(str(expected2), str(result2)))
@pytest.mark.parametrize(
"inp,expected",
[
([[1], []], [[1], []]), # don't move unnecessarily
([[0, 0], []], [[0], [0]]), # balance
([[0.1, 0.1], []], [[0], [0]]), # balance even if results in even
([[0, 0, 0], []], [[0, 0], [0]]), # don't over balance
([[0, 0], [0, 0, 0], []], [[0, 0], [0, 0], [0]]), # move from larger
([[0, 0, 0], [0], []], [[0, 0], [0], [0]]), # move to smaller
([[0, 1], []], [[1], [0]]), # choose easier first
([[0, 0, 0, 0], [], []], [[0, 0], [0], [0]]), # spread evenly
([[1, 0, 2, 0], [], []], [[2, 1], [0], [0]]), # move easier
([[1, 1, 1], []], [[1, 1], [1]]), # be willing to move costly items
([[1, 1, 1, 1], []], [[1, 1, 1], [1]]), # but don't move too many
(
[[0, 0], [0, 0], [0, 0], []], # no one clearly saturated
[[0, 0], [0, 0], [0], [0]],
),
(
[[4, 2, 2, 2, 2, 1, 1], [4, 2, 1, 1], [], [], []],
[[4, 2, 2, 2, 2], [4, 2, 1], [1], [1], [1]],
),
pytest.param(
[[1, 1, 1, 1, 1, 1, 1], [1, 1], [1, 1], [1, 1], []],
[[1, 1, 1, 1, 1], [1, 1], [1, 1], [1, 1], [1, 1]],
marks=pytest.mark.xfail(
reason="Some uncertainty based on executing stolen task"
),
),
],
)
def test_balance(inp, expected):
async def test(*args, **kwargs):
await assert_balanced(inp, expected, *args, **kwargs)
test = gen_cluster(
client=True,
nthreads=[("127.0.0.1", 1)] * len(inp),
config={
"distributed.scheduler.default-task-durations": {
str(i): 1 for i in range(10)
}
},
)(test)
test()
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2, Worker=Nanny, timeout=20)
async def test_restart(c, s, a, b):
futures = c.map(
slowinc, range(100), delay=0.1, workers=a.address, allow_other_workers=True
)
while not s.processing[b.worker_address]:
await asyncio.sleep(0.01)
steal = s.extensions["stealing"]
assert any(st for st in steal.stealable_all)
assert any(x for L in steal.stealable.values() for x in L)
await c.restart(timeout=10)
assert not any(x for x in steal.stealable_all)
assert not any(x for L in steal.stealable.values() for x in L)
@gen_cluster(
client=True,
config={"distributed.scheduler.default-task-durations": {"slowadd": 0.001}},
)
async def test_steal_communication_heavy_tasks(c, s, a, b):
steal = s.extensions["stealing"]
x = c.submit(mul, b"0", int(s.bandwidth), workers=a.address)
y = c.submit(mul, b"1", int(s.bandwidth), workers=b.address)
futures = [
c.submit(
slowadd,
x,
y,
delay=1,
pure=False,
workers=a.address,
allow_other_workers=True,
)
for i in range(10)
]
while not any(f.key in s.rprocessing for f in futures):
await asyncio.sleep(0.01)
steal.balance()
while steal.in_flight:
await asyncio.sleep(0.001)
assert s.processing[b.address]
@gen_cluster(client=True)
async def test_steal_twice(c, s, a, b):
x = c.submit(inc, 1, workers=a.address)
await wait(x)
futures = [c.submit(slowadd, x, i, delay=0.2) for i in range(100)]
while len(s.tasks) < 100: # tasks are all allocated
await asyncio.sleep(0.01)
# Army of new workers arrives to help
workers = await asyncio.gather(*[Worker(s.address, loop=s.loop) for _ in range(20)])
await wait(futures)
has_what = dict(s.has_what) # take snapshot
empty_workers = [w for w, keys in has_what.items() if not len(keys)]
if len(empty_workers) > 2:
pytest.fail(
"Too many workers without keys (%d out of %d)"
% (len(empty_workers), len(has_what))
)
assert max(map(len, has_what.values())) < 30
await c._close()
await asyncio.gather(*[w.close() for w in workers])
@gen_cluster(client=True)
async def test_dont_steal_executing_tasks(c, s, a, b):
steal = s.extensions["stealing"]
future = c.submit(slowinc, 1, delay=0.5, workers=a.address)
while not a.executing_count:
await asyncio.sleep(0.01)
steal.move_task_request(
s.tasks[future.key], s.workers[a.address], s.workers[b.address]
)
await asyncio.sleep(0.1)
assert a.tasks[future.key].state == "executing"
assert not b.executing_count
@gen_cluster(client=True)
async def test_dont_steal_already_released(c, s, a, b):
future = c.submit(slowinc, 1, delay=0.05, workers=a.address)
key = future.key
await asyncio.sleep(0.05)
assert key in a.tasks
del future
await asyncio.sleep(0.05)
# In case the system is slow (e.g. network) ensure that nothing bad happens
# if the key was already released
assert key not in a.tasks
a.steal_request(key)
assert a.batched_stream.buffer == [
{"op": "steal-response", "key": key, "state": None}
]
with captured_logger(
logging.getLogger("distributed.stealing"), level=logging.DEBUG
) as stealing_logs:
await asyncio.sleep(0.05)
logs = stealing_logs.getvalue()
assert f"Key released between request and confirm: {key}" in logs
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
async def test_dont_steal_long_running_tasks(c, s, a, b):
def long(delay):
with worker_client() as c:
sleep(delay)
await c.submit(long, 0.1) # learn duration
await c.submit(inc, 1) # learn duration
long_tasks = c.map(long, [0.5, 0.6], workers=a.address, allow_other_workers=True)
while sum(map(len, s.processing.values())) < 2: # let them start
await asyncio.sleep(0.01)
start = time()
while any(t.key in s.extensions["stealing"].key_stealable for t in long_tasks):
await asyncio.sleep(0.01)
assert time() < start + 1
na = a.executing_count
nb = b.executing_count
incs = c.map(inc, range(100), workers=a.address, allow_other_workers=True)
await asyncio.sleep(0.2)
await wait(long_tasks)
for t in long_tasks:
assert (
sum(log[1] == "executing" for log in a.story(t))
+ sum(log[1] == "executing" for log in b.story(t))
) <= 1
@pytest.mark.xfail(
sys.version_info[:2] == (3, 8),
reason="Sporadic failure on Python 3.8",
strict=False,
)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 5)] * 2)
async def test_cleanup_repeated_tasks(c, s, a, b):
class Foo:
pass
s.extensions["stealing"]._pc.callback_time = 20
await c.submit(slowidentity, -1, delay=0.1)
objects = [c.submit(Foo, pure=False, workers=a.address) for _ in range(50)]
x = c.map(
slowidentity, objects, workers=a.address, allow_other_workers=True, delay=0.05
)
del objects
await wait(x)
assert a.data and b.data
assert len(a.data) + len(b.data) > 10
ws = weakref.WeakSet()
ws.update(a.data.values())
ws.update(b.data.values())
del x
start = time()
while a.data or b.data:
await asyncio.sleep(0.01)
assert time() < start + 1
assert not s.who_has
assert not any(s.has_what.values())
assert not list(ws)
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
async def test_lose_task(c, s, a, b):
with captured_logger("distributed.stealing") as log:
s.periodic_callbacks["stealing"].interval = 1
for i in range(100):
futures = c.map(
slowinc,
range(10),
delay=0.01,
pure=False,
workers=a.address,
allow_other_workers=True,
)
await asyncio.sleep(0.01)
del futures
out = log.getvalue()
assert "Error" not in out
@gen_cluster(client=True)
async def test_worker_stealing_interval(c, s, a, b):
from distributed.scheduler import WorkStealing
ws = WorkStealing(s)
assert ws._pc.callback_time == 100
with dask.config.set({"distributed.scheduler.work-stealing-interval": "500ms"}):
ws = WorkStealing(s)
assert ws._pc.callback_time == 500
# Default unit is `ms`
with dask.config.set({"distributed.scheduler.work-stealing-interval": 2}):
ws = WorkStealing(s)
assert ws._pc.callback_time == 2
@gen_cluster(client=True)
async def test_balance_with_longer_task(c, s, a, b):
np = pytest.importorskip("numpy")
await c.submit(slowinc, 0, delay=0) # scheduler learns that slowinc is very fast
x = await c.scatter(np.arange(10000), workers=[a.address])
y = c.submit(
slowinc, 1, delay=5, workers=[a.address], priority=1
) # a surprisingly long task
z = c.submit(
inc, x, workers=[a.address], allow_other_workers=True, priority=0
) # a task after y, suggesting a, but open to b
await z
assert z.key in b.data
|