def test_prefer_broker_nodes(): """ b0 b1 b2 | \ / a0 a1 a1 should be run before a0 """ a, b, c = 'abc' dsk = {(a, 0): (f,), (a, 1): (f,), (b, 0): (f, (a, 0)), (b, 1): (f, (a, 1)), (b, 2): (f, (a, 1))} dependencies, dependents = get_deps(dsk) nd = ndependents(dependencies, dependents) cm = child_max(dependencies, dependents, nd) o = order(dsk) assert o[(a, 1)] < o[(a, 0)] # Switch name of 0, 1 to ensure that this isn't due to string comparison dsk = {(a, 0): (f,), (a, 1): (f,), (b, 0): (f, (a, 0)), (b, 1): (f, (a, 1)), (b, 2): (f, (a, 0))} o = order(dsk) assert o[(a, 1)] > o[(a, 0)]
def test_nearest_neighbor(abcde): """ a1 a2 a3 a4 a5 a6 a7 a8 a9 \ | / \ | / \ | / \ | / b1 b2 b3 b4 Want to finish off a local group before moving on. This is difficult because all groups are connected. """ a, b, c, _, _ = abcde a1, a2, a3, a4, a5, a6, a7, a8, a9 = [a + i for i in '123456789'] b1, b2, b3, b4 = [b + i for i in '1234'] dsk = {b1: (f,), b2: (f,), b3: (f,), b4: (f,), a1: (f, b1), a2: (f, b1), a3: (f, b1, b2), a4: (f, b2), a5: (f, b2, b3), a6: (f, b3), a7: (f, b3, b4), a8: (f, b4), a9: (f, b4)} o = order(dsk) assert 3 < sum(o[a + i] < len(o) / 2 for i in '123456789') < 7 assert 1 < sum(o[b + i] < len(o) / 2 for i in '1234') < 4 assert o[min([b1, b2, b3, b4])] == 0
def test_finish_task(): dsk = {'x': 1, 'y': 2, 'z': (inc, 'x'), 'w': (add, 'z', 'y')} sortkey = order(dsk).get state = start_state_from_dask(dsk) state['ready'].remove('z') state['running'] = set(['z', 'other-task']) task = 'z' result = 2 state['cache']['z'] = result finish_task(dsk, task, state, set(), sortkey) assert state == {'cache': {'y': 2, 'z': 2}, 'dependencies': {'w': set(['y', 'z']), 'x': set([]), 'y': set([]), 'z': set(['x'])}, 'finished': set(['z']), 'released': set(['x']), 'running': set(['other-task']), 'dependents': {'w': set([]), 'x': set(['z']), 'y': set(['w']), 'z': set(['w'])}, 'ready': ['w'], 'waiting': {}, 'waiting_data': {'y': set(['w']), 'z': set(['w'])}}
def test_base_of_reduce_preferred(): """ a3 /| a2 | /| | a1 | | /| | | a0 | | | | | | | b0 b1 b2 b3 \ \ / / c We really want to run b0 quickly """ dsk = dict((('a', i), (f, ('a', i - 1), ('b', i))) for i in [1, 2, 3]) dsk[('a', 0)] = (f, ('b', 0)) dsk.update(dict((('b', i), (f, 'c', 1)) for i in [0, 1, 2, 3])) dsk['c'] = 1 o = order(dsk) assert o == {('a', 3): 0, ('a', 2): 1, ('a', 1): 2, ('a', 0): 3, ('b', 0): 4, 'c': 5, ('b', 1): 6, ('b', 2): 7, ('b', 3): 8} # ('b', 0) is the most important out of ('b', i) assert min([('b', i) for i in [0, 1, 2, 3]], key=o.get) == ('b', 0)
def test_string_ordering(): """ Prefer ordering tasks by name first """ dsk = {('a', 1): (f,), ('a', 2): (f,), ('a', 3): (f,)} o = order(dsk) assert o == {('a', 1): 0, ('a', 2): 1, ('a', 3): 2}
def test_base_of_reduce_preferred(abcde): """ a3 /| a2 | /| | a1 | | /| | | a0 | | | | | | | b0 b1 b2 b3 \ \ / / c We really want to run b0 quickly """ a, b, c, d, e = abcde dsk = {(a, i): (f, (a, i - 1), (b, i)) for i in [1, 2, 3]} dsk[(a, 0)] = (f, (b, 0)) dsk.update({(b, i): (f, c, 1) for i in [0, 1, 2, 3]}) dsk[c] = 1 o = order(dsk) assert o[(b, 0)] <= 4 assert o[(b, 1)] <= 6
def update_graph(self, dsk=None, keys=None, restrictions={}): update_state(self.dask, self.dependencies, self.dependents, self.held_data, self.who_has, self.in_play, self.waiting, self.waiting_data, dsk, keys) cover_aliases(self.dask, dsk) self.restrictions.update(restrictions) new_keyorder = order(dsk) # TODO: define order wrt old graph for key in new_keyorder: if key not in self.keyorder: # TODO: add test for this self.keyorder[key] = (self.generation, new_keyorder[key]) # prefer old if len(dsk) > 1: self.generation += 1 # older graph generations take precedence for key in dsk: for dep in self.dependencies[key]: if dep in self.exceptions_blame: self.mark_failed(key, self.exceptions_blame[dep]) self.seed_ready_tasks(dsk) for key in keys: if self.who_has[key]: self.mark_key_in_memory(key) for plugin in self.plugins[:]: try: plugin.update_graph(self, dsk, keys, restrictions) except Exception as e: logger.exception(e)
def test_avoid_upwards_branching_complex(abcde): """ a1 | e2 a2 d2 d3 | | \ / e1 a3 d1 \ / \ / b1 c1 | | b2 c2 | c3 Prefer c1 over b1 because c1 will stay in memory less long while b1 computes """ a, b, c, d, e = abcde dsk = {(a, 1): (f, (a, 2)), (a, 2): (f, (a, 3)), (a, 3): (f, (b, 1), (c, 1)), (b, 1): (f, (b, 2)), (b, 2): (f,), (c, 1): (f, (c, 2)), (c, 2): (f, (c, 3)), (c, 3): (f,), (d, 1): (f, (c, 1)), (d, 2): (f, (d, 1)), (d, 3): (f, (d, 1)), (e, 1): (f, (b, 1)), (e, 2): (f, (e, 1))} o = order(dsk) assert o[(c, 1)] < o[(b, 1)]
def test_avoid_upwards_branching_complex(): """ a1 | e2 a2 d2 d3 | | \ / e1 a3 d1 \ / \ / b1 c1 | | b2 c2 | c3 Prefer b1 over c1 because it will be easier to finish its dependents """ dsk = {(a, 1): (f, (a, 2)), (a, 2): (f, (a, 3)), (a, 3): (f, (b, 1), (c, 1)), (b, 1): (f, (b, 2)), (c, 1): (f, (c, 2)), (c, 2): (f, (c, 3)), (d, 1): (f, (c, 1)), (d, 2): (f, (d, 1)), (d, 3): (f, (d, 1)), (e, 1): (f, (b, 1)), (e, 2): (f, (e, 1))} o = order(dsk) assert o[(b, 1)] < o[(c, 1)]
def test_avoid_upwards_branching(): """ a1 | a2 | a3 d1 / \ / b1 c1 | | b2 c2 | c3 Prefer b1 over c1 because it won't stick around waiting for d1 to complete """ dsk = {(a, 1): (f, (a, 2)), (a, 2): (f, (a, 3)), (a, 3): (f, (b, 1), (c, 1)), (b, 1): (f, (b, 2)), (c, 1): (f, (c, 2)), (c, 2): (f, (c, 3)), (d, 1): (f, (c, 1))} o = order(dsk) assert o[(b, 1)] < o[(c, 1)]
def test_ordering_prefers_tasks_that_release_data(): a, b, c = 'abc' f = lambda *args: None d = {(a, 0): (f,), (a, 1): (f,), (b, 0): (f, (a, 0)), (b, 1): (f, (a, 1)), (b, 2): (f, (a, 1))} o = order(d) assert o[(a, 1)] > o[(a, 0)] d = {(a, 0): (f,), (a, 1): (f,), (b, 0): (f, (a, 0)), (b, 1): (f, (a, 1)), (b, 2): (f, (a, 0))} o = order(d) assert o[(a, 1)] < o[(a, 0)]
def test_ordering_keeps_groups_together(): d = dict(((a, i), (f,)) for i in range(4)) d.update({(b, 0): (f, (a, 0), (a, 1)), (b, 1): (f, (a, 2), (a, 3))}) o = order(d) assert abs(o[(a, 0)] - o[(a, 1)]) == 1 assert abs(o[(a, 2)] - o[(a, 3)]) == 1 d = dict(((a, i), (f,)) for i in range(4)) d.update({(b, 0): (f, (a, 0), (a, 2)), (b, 1): (f, (a, 1), (a, 3))}) o = order(d) assert abs(o[(a, 0)] - o[(a, 2)]) == 1 assert abs(o[(a, 1)] - o[(a, 3)]) == 1
def test_local_parents_of_reduction(abcde): """ c1 | b1 c2 | /| a1 b2 c3 | /| a2 b3 | a3 Prefer to finish a1 stack before proceding to b2 """ a, b, c, d, e = abcde a1, a2, a3 = [a + i for i in '123'] b1, b2, b3 = [b + i for i in '123'] c1, c2, c3 = [c + i for i in '123'] expected = [a3, a2, a1, b3, b2, b1, c3, c2, c1] log = [] def f(x): def _(*args): log.append(x) return _ dsk = {a3: (f(a3),), a2: (f(a2), a3), a1: (f(a1), a2), b3: (f(b3),), b2: (f(b2), b3, a2), b1: (f(b1), b2), c3: (f(c3),), c2: (f(c2), c3, b2), c1: (f(c1), c2)} order(dsk) dask.get(dsk, [a1, b1, c1]) # trigger computation assert log == expected
def test_string_ordering_dependents(): """ Prefer ordering tasks by name first even when in dependencies """ dsk = {('a', 1): (f, 'b'), ('a', 2): (f, 'b'), ('a', 3): (f, 'b'), 'b': (f,)} o = order(dsk) assert o == {'b': 0, ('a', 1): 1, ('a', 2): 2, ('a', 3): 3}
def test_prefer_short_ancestor(abcde): """ From https://github.com/dask/dask-ml/issues/206#issuecomment-395869929 Two cases, one where chunks of an array are independent, and one where the chunks of an array have a shared source. We handled the independent one "well" earlier. Good: c2 / \ \ / \ \ c1 \ \ / | \ \ \ c0 a0 b0 a1 b1 Bad: c2 / \ \ / \ \ c1 \ \ / | \ \ \ c0 a0 b0 a1 b1 \ \ / / \ \ / / a-b The difference is that all the `a` and `b` tasks now have a common ancestor. We would like to choose c1 *before* a1, and b1 because * we can release a0 and b0 once c1 is done * we don't need a1 and b1 to compute c1. """ a, b, c, _, _ = abcde ab = a + b dsk = { ab: 0, (a, 0): (f, ab, 0, 0), (b, 0): (f, ab, 0, 1), (c, 0): 0, (c, 1): (f, (c, 0), (a, 0), (b, 0)), (a, 1): (f, ab, 1, 0), (b, 1): (f, ab, 1, 1), (c, 2): (f, (c, 1), (a, 1), (b, 1)), } o = order(dsk) assert o[(b, 0)] < o[(b, 1)] assert o[(b, 0)] < o[(c, 2)] assert o[(c, 1)] < o[(c, 2)] assert o[(c, 1)] < o[(a, 1)]
def test_break_ties_by_str(): dsk = {('x', i): (inc, i) for i in range(10)} x_keys = sorted(dsk) dsk['y'] = list(x_keys) o = order(dsk) expected = {'y': 0} expected.update({k: i + 1 for i, k in enumerate(x_keys)}) assert o == expected
def test_ordering_prefers_depth_first(): a, b, c = 'abc' f = lambda *args: None d = {(a, 0): (f,), (b, 0): 0, (c, 0): (f,), (a, 1): (f,), (b, 1): (f, (a, 0), (b, 0), (c, 0)), (c, 1): (f,), (a, 2): (f,), (b, 2): (f, (a, 1), (b, 1), (c, 1)), (c, 2): (f,), (a, 3): (f,), (b, 3): (f, (a, 2), (b, 2), (c, 2)), (c, 3): (f,)} o = order(d) assert issorted(list(map(o.get, [(a, i) for i in range(4)])), reverse=True) assert issorted(list(map(o.get, [(c, i) for i in range(4)])), reverse=True)
def update_graph(self, dsk=None, keys=None, restrictions=None, loose_restrictions=None): """ Add new computations to the internal dask graph This happens whenever the Executor calls submit, map, get, or compute. """ for k in list(dsk): if dsk[k] is k: del dsk[k] update_state( self.dask, self.dependencies, self.dependents, self.held_data, self.who_has, self.in_play, self.waiting, self.waiting_data, dsk, keys, ) cover_aliases(self.dask, dsk) if restrictions: restrictions = {k: set(map(ensure_ip, v)) for k, v in restrictions.items()} self.restrictions.update(restrictions) if loose_restrictions: self.loose_restrictions |= loose_restrictions new_keyorder = order(dsk) # TODO: define order wrt old graph for key in new_keyorder: if key not in self.keyorder: # TODO: add test for this self.keyorder[key] = (self.generation, new_keyorder[key]) # prefer old if len(dsk) > 1: self.generation += 1 # older graph generations take precedence for key in dsk: for dep in self.dependencies[key]: if dep in self.exceptions_blame: self.mark_failed(key, self.exceptions_blame[dep]) self.seed_ready_tasks(dsk) for key in keys: if self.who_has[key]: self.mark_key_in_memory(key) for plugin in self.plugins[:]: try: plugin.update_graph(self, dsk, keys, restrictions or {}) except Exception as e: logger.exception(e)
def test_gh_3055(): da = pytest.importorskip('dask.array') A, B = 20, 99 x = da.random.normal(size=(A, B), chunks=(1, None)) for _ in range(2): y = (x[:, None, :] * x[:, :, None]).cumsum(axis=0) x = x.cumsum(axis=0) w = (y * x[:, None]).sum(axis=(1,2)) dsk = dict(w.__dask_graph__()) o = order(dsk) L = [o[k] for k in w.__dask_keys__()] assert sorted(L) == L
def test_prefer_deep(): """ c | y b | | x a Prefer longer chains first so we should start with c """ dsk = {a: 1, b: (f, a), c: (f, b), 'x': 1, 'y': (f, 'x')} o = order(dsk) assert o == {c: 0, b: 1, a: 2, 'y': 3, 'x': 4}
def test_avoid_broker_nodes(abcde): """ b0 b1 b2 | \ / a0 a1 a0 should be run before a1 """ a, b, c, d, e = abcde dsk = {(a, 0): (f,), (a, 1): (f,), (b, 0): (f, (a, 0)), (b, 1): (f, (a, 1)), (b, 2): (f, (a, 1))} o = order(dsk) assert o[(a, 0)] < o[(a, 1)] # Switch name of 0, 1 to ensure that this isn't due to string comparison dsk = {(a, 1): (f,), (a, 0): (f,), (b, 0): (f, (a, 1)), (b, 1): (f, (a, 0)), (b, 2): (f, (a, 0))} o = order(dsk) assert o[(a, 0)] > o[(a, 1)]
def test_prefer_broker_nodes(): """ b0 b1 b2 | \ / a0 a1 a1 should be run before a0 """ a, b, c = 'abc' dsk = {(a, 0): (f,), (a, 1): (f,), (b, 0): (f, (a, 0)), (b, 1): (f, (a, 1)), (b, 2): (f, (a, 1))} o = order(dsk) assert o[(a, 1)] < o[(a, 0)] # Switch name of 0, 1 to ensure that this isn't due to string comparison dsk = {(a, 0): (f,), (a, 1): (f,), (b, 0): (f, (a, 0)), (b, 1): (f, (a, 1)), (b, 2): (f, (a, 0))} o = order(dsk) assert o[(a, 1)] > o[(a, 0)]
def test_prefer_deep(): """ c | y b | | x a Prefer longer chains first so we should start with c """ dsk = {'a': 1, 'b': (f, 'a'), 'c': (f, 'b'), 'x': 1, 'y': (f, 'x')} o = order(dsk) assert o == {'c': 0, 'b': 1, 'a': 2, 'y': 3, 'x': 4}
def test_prefer_short_narrow(abcde): # See test_prefer_short_ancestor for a fail case. a, b, c, _, _ = abcde dsk = { (a, 0): 0, (b, 0): 0, (c, 0): 0, (c, 1): (f, (c, 0), (a, 0), (b, 0)), (a, 1): 1, (b, 1): 1, (c, 2): (f, (c, 1), (a, 1), (b, 1)), } o = order(dsk) assert o[(b, 0)] < o[(b, 1)] assert o[(b, 0)] < o[(c, 2)] assert o[(c, 1)] < o[(c, 2)]
def test_deep_bases_win_over_dependents(): """ d should come before e and probably before one of b and c a / | \ . b c | / \ | / e d """ dsk = {'a': (f, 'b', 'c', 'd'), 'b': (f, 'd', 'e'), 'c': (f, 'd'), 'd': 1, 'e': 2} o = order(dsk) assert o['d'] < o['e'] assert o['d'] < o['b'] or o['d'] < o['c']
def test_prefer_deep(abcde): """ c | e b | | d a Prefer longer chains first so we should start with c """ a, b, c, d, e = abcde dsk = {a: 1, b: (f, a), c: (f, b), d: 1, e: (f, d)} o = order(dsk) assert o[a] < o[d] assert o[b] < o[d]
def test_gh_3055(): da = pytest.importorskip('dask.array') A, B = 20, 99 orig = x = da.random.normal(size=(A, B), chunks=(1, None)) for _ in range(2): y = (x[:, None, :] * x[:, :, None]).cumsum(axis=0) x = x.cumsum(axis=0) w = (y * x[:, None]).sum(axis=(1,2)) dsk = dict(w.__dask_graph__()) o = order(dsk) L = [o[k] for k in w.__dask_keys__()] assert sum(x < len(o) / 2 for x in L) > len(L) / 3 # some complete quickly L = [o[k] for kk in orig.__dask_keys__() for k in kk] assert sum(x > len(o) / 2 for x in L) > len(L) / 3 # some start later assert sorted(L) == L # operate in order
def test_prefer_short_dependents(abcde): """ a | d b e \ | / c Prefer to finish d and e before starting b. That way c can be released during the long computations. """ a, b, c, d, e = abcde dsk = {c: (f,), d: (f, c), e: (f, c), b: (f, c), a: (f, b)} o = order(dsk) assert o[d] < o[b] assert o[e] < o[b]
def test_deep_bases_win_over_dependents(): """ It's not clear who should run first, e or d 1. d is nicer because it exposes parallelism 2. e is nicer (hypothetically) because it will be sooner released (though in this we need d to run first regardless) a / | \ . b c | / \ | / e d """ dsk = {a: (f, b, c, d), b: (f, d, e), c: (f, d), d: 1, e: 2} o = order(dsk) assert o[e] < o[d] assert o[d] < o[b] or o[d] < o[c]
def test_deep_bases_win_over_dependents(): """ d should come before e and probably before one of b and c a / | \ . b c | / \ | / e d """ dsk = {'a': (f, 'b', 'c', 'd'), 'b': (f, 'd', 'e'), 'c': (f, 'd'), 'd': 1, 'e': 2} dependencies, dependents = get_deps(dsk) nd = ndependents(dependencies, dependents) cm = child_max(dependencies, dependents, nd) o = order(dsk) assert o['d'] < o['e'] assert o['d'] < o['b'] or o['d'] < o['c']
def test_order_doesnt_fail_on_mixed_type_keys(abcde): order({"x": (inc, 1), ("y", 0): (inc, 2), "z": (add, "x", ("y", 0))})
def test_order_with_equal_dependents(abcde): """From https://github.com/dask/dask/issues/5859#issuecomment-608422198 See the visualization of `(maxima, argmax)` example from the above comment. This DAG has enough structure to exercise more parts of `order` """ a, b, c, d, e = abcde dsk = {} abc = [a, b, c, d] for x in abc: dsk.update({ (x, 0): 0, (x, 1): (f, (x, 0)), (x, 2, 0): (f, (x, 0)), (x, 2, 1): (f, (x, 1)), }) for i, y in enumerate(abc): dsk.update({ (x, 3, i): (f, (x, 2, 0), (y, 2, 1)), # cross x and y (x, 4, i): (f, (x, 3, i)), (x, 5, i, 0): (f, (x, 4, i)), (x, 5, i, 1): (f, (x, 4, i)), (x, 6, i, 0): (f, (x, 5, i, 0)), (x, 6, i, 1): (f, (x, 5, i, 1)), }) o = order(dsk) total = 0 for x in abc: for i in range(len(abc)): val = o[(x, 6, i, 1)] - o[(x, 6, i, 0)] assert val > 0 # ideally, val == 2 total += val assert total <= 32 # ideally, this should be 2 * 16 = 32 # Add one to the end of the nine bundles dsk2 = dict(dsk) for x in abc: for i in range(len(abc)): dsk2[(x, 7, i, 0)] = (f, (x, 6, i, 0)) o = order(dsk2) total = 0 for x in abc: for i in range(len(abc)): val = o[(x, 7, i, 0)] - o[(x, 6, i, 1)] assert val > 0 # ideally, val == 3 total += val assert total <= 165 # ideally, this should be 3 * 16 == 48 # Remove one from each of the nine bundles dsk3 = dict(dsk) for x in abc: for i in range(len(abc)): del dsk3[(x, 6, i, 1)] o = order(dsk3) total = 0 for x in abc: for i in range(len(abc)): val = o[(x, 6, i, 0)] - o[(x, 5, i, 1)] assert val > 0 # ideally, val == 2 total += val assert total <= 119 # ideally, this should be 2 * 16 == 32 # Remove another one from each of the nine bundles dsk4 = dict(dsk3) for x in abc: for i in range(len(abc)): del dsk4[(x, 6, i, 0)] o = order(dsk4) total = 0 for x in abc: for i in range(len(abc)): assert o[(x, 5, i, 1)] - o[(x, 5, i, 0)] == 1
def test_order_empty(): assert order({}) == {}
def test_type_comparisions_ok(abcde): a, b, c, d, e = abcde dsk = {a: 1, (a, 1): 2, (a, b, 1): 3} order(dsk) # this doesn't err
def test_string_ordering(): """ Prefer ordering tasks by name first """ dsk = {("a", 1): (f, ), ("a", 2): (f, ), ("a", 3): (f, )} o = order(dsk) assert o == {("a", 1): 0, ("a", 2): 1, ("a", 3): 2}
def scheduler(scheduler_queue, report_queue, worker_queues, delete_queue, who_has, has_what, ncores, dsk=None, restrictions=None, waiting=None, stacks=None, processing=None): """ The scheduler coroutine for dask scheduling This coroutine manages interactions with all worker cores and with the delete coroutine through queues. Parameters ---------- scheduler_queue: tornado.queues.Queue Get information from outside report_queue: tornado.queues.Queue Report tasks done worker_queues: dict {worker: tornado.queues.Queue} One queue per worker node. Each queue is listened to by several worker_core coroutiens. delete_queue: tornado.queues.Queue One queue listened to by ``delete`` which connects to the center to delete unnecessary intermediate data who_has: dict {key: set} Mapping key to {set of worker-identities} has_what: dict {worker: set} Mapping worker-identity to {set of keys} ncores: dict {worker: int} Mapping worker-identity to number-of-cores """ held_data = set() if dsk is None: dsk = dict() if restrictions is None: restrictions = dict() if waiting is None: waiting = dict() if stacks is None: stacks = dict() stacks.update({worker: list() for worker in ncores}) if processing is None: processing = dict() processing.update({worker: set() for worker in ncores}) dependencies = dict() dependents = dict() waiting_data = dict() in_play = set(who_has) # keys in memory, stacks, processing, or waiting released = set() keyorder = dict() generation = 0 @gen.coroutine def cleanup(): """ Clean up queues and coroutines, prepare to stop """ n = 0 delete_queue.put_nowait({'op': 'close'}) n += 1 for w, nc in ncores.items(): for i in range(nc): worker_queues[w].put_nowait({'op': 'close'}) n += 1 for i in range(n): yield scheduler_queue.get() def mark_ready_to_run(key): """ Send task to an appropriate worker, trigger worker if idle """ if key in waiting: del waiting[key] new_worker = decide_worker(dependencies, stacks, who_has, restrictions, key) stacks[new_worker].append(key) ensure_occupied(new_worker) def ensure_occupied(worker): """ If worker is free, spin up a task on that worker """ logger.debug('Ensure worker is occupied: %s', worker) while stacks[worker] and ncores[worker] > len(processing[worker]): key = stacks[worker].pop() processing[worker].add(key) logger.debug("Send job to worker: %s, %s, %s", worker, key, dsk[key]) worker_queues[worker].put_nowait({ 'op': 'compute-task', 'key': key, 'task': dsk[key], 'needed': dependencies[key] }) def seed_ready_tasks(): """ Distribute leaves among workers """ new_stacks = assign_many_tasks( dependencies, waiting, keyorder, who_has, stacks, restrictions, [k for k, deps in waiting.items() if not deps]) logger.debug("Seed ready tasks: %s", new_stacks) for worker, stack in new_stacks.items(): if stack: ensure_occupied(worker) def release_key(key): if key not in held_data and not waiting_data.get(key): delete_queue.put_nowait({'op': 'delete-task', 'key': key}) for w in who_has[key]: has_what[w].remove(key) del who_has[key] if key in waiting_data: del waiting_data[key] if key in in_play: in_play.remove(key) def debug_state(msg=None): if msg: logger.debug(msg) logger.debug( '\n\nwaiting: %s\n\nstacks: %s\n\nprocessing: %s\n\n' 'in_play: %s\n\n', waiting, stacks, processing, in_play) my_heal_missing_data = partial(heal_missing_data, dsk, dependencies, dependents, held_data, who_has, in_play, waiting, waiting_data) while True: msg = yield scheduler_queue.get() if msg['op'] == 'close': break elif msg['op'] == 'update-graph': new_dsk = msg['dsk'] new_keys = msg['keys'] update_state(dsk, dependencies, dependents, held_data, who_has, in_play, waiting, waiting_data, new_dsk, new_keys) restrictions.update(msg.get('restrictions', {})) new_keyorder = order(new_dsk) for key in new_keyorder: if key not in keyorder: # TODO: add test for this keyorder[key] = (generation, new_keyorder[key] ) # prefer old if len(new_dsk) > 1: generation += 1 # older graph generations take precedence seed_ready_tasks() for key in new_keys: if who_has[key]: report_queue.put_nowait({ 'op': 'task-finished', 'worker': first(who_has[key]), 'key': key }) elif msg['op'] == 'task-finished': key = msg['key'] worker = msg['worker'] logger.debug("task finished: %s, %s", key, worker) who_has[key].add(worker) has_what[worker].add(key) with ignoring(KeyError): processing[worker].remove(key) report_queue.put_nowait(msg) for dep in sorted(dependents[key], key=keyorder.get, reverse=True): if dep in waiting: s = waiting[dep] with ignoring(KeyError): s.remove(key) if not s: # new task ready to run mark_ready_to_run(dep) for dep in dependencies[key]: if dep in waiting_data: s = waiting_data[dep] with ignoring(KeyError): s.remove(key) if not s and dep: release_key(dep) ensure_occupied(worker) elif msg['op'] == 'task-erred': processing[msg['worker']].remove(msg['key']) in_play.remove(msg['key']) report_queue.put_nowait(msg) ensure_occupied(msg['worker']) elif msg['op'] in ('missing-data', 'task-missing-data'): missing = set(msg['missing']) logger.debug("Recovering missing data: %s", missing) for k in missing: with ignoring(KeyError): workers = who_has.pop(k) for worker in workers: has_what[worker].remove(k) my_heal_missing_data(missing) if msg['op'] == 'task-missing-data': key = msg['key'] with ignoring(KeyError): processing[msg['worker']].remove(key) waiting[key] = missing logger.info('task missing data, %s, %s', key, waiting) with ignoring(KeyError): processing[msg['worker']].remove(msg['key']) ensure_occupied(msg['worker']) seed_ready_tasks() elif msg['op'] == 'worker-failed': worker = msg['worker'] keys = has_what.pop(worker) del worker_queues[worker] del ncores[worker] del stacks[worker] del processing[worker] missing_keys = set() for key in keys: who_has[key].remove(worker) if not who_has[key]: missing_keys.add(key) gone_data = {k for k, v in who_has.items() if not v} in_play -= missing_keys for k in gone_data: del who_has[k] state = heal(dependencies, dependents, set(who_has), stacks, processing, waiting, waiting_data) waiting_data = state['waiting_data'] waiting = state['waiting'] released = state['released'] in_play = state['in_play'] add_keys = {k for k, v in waiting.items() if not v} for key in held_data & released: report_queue.put_nowait({'op': 'lost-key', 'key': key}) for key in add_keys: mark_ready_to_run(key) for key in set(who_has) & released - held_data: delete_queue.put_nowait({'op': 'delete-task', 'key': key}) elif msg['op'] == 'release-held-data': if msg['key'] in held_data: logger.debug("Release key: %s", msg['key']) held_data.remove(msg['key']) release_key(msg['key']) else: logger.warn("Bad message: %s", msg) logger.debug('Finished scheduling') yield cleanup()