def test_failed_worker_without_warning(c, s, a, b): L = c.map(inc, range(10)) yield _wait(L) original_process = a.process a.process.terminate() start = time() while a.process is original_process and not isalive(a.process): yield gen.sleep(0.01) assert time() - start < 10 yield gen.sleep(0.5) start = time() while len(s.ncores) < 2: yield gen.sleep(0.01) assert time() - start < 10 yield _wait(L) L2 = c.map(inc, range(10, 20)) yield _wait(L2) assert all(len(keys) > 0 for keys in s.has_what.values()) ncores2 = s.ncores.copy() yield c._restart() L = c.map(inc, range(10)) yield _wait(L) assert all(len(keys) > 0 for keys in s.has_what.values()) assert not (set(ncores2) & set(s.ncores)) # no overlap
def test_steal_cheap_data_slow_computation(c, s, a, b): x = c.submit(slowinc, 100, delay=0.1) # learn that slowinc is slow yield _wait([x]) futures = c.map(slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True) yield _wait(futures) assert abs(len(a.data) - len(b.data)) <= 5
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) yield _wait([x]) future = c.submit(np.sum, [1], workers=a.address) # learn that sum is fast yield _wait([future]) cheap = [c.submit(np.sum, x, pure=False, workers=a.address, allow_other_workers=True) for i in range(10)] yield _wait(cheap) assert len(b.data) == 0 assert len(a.data) == 12
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 yield _wait(x) s.task_duration['slowidentity'] = 0.2 futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(2)] yield _wait(futures) assert len(a.data) == 3 assert not any(w.task_state for w in rest)
def test_persist(c, s, a, b): x = delayed(inc)(1) x2, = persist(x) yield _wait(x2) assert x2.key in a.data or x2.key in b.data y = delayed(inc)(10) y2, one = persist(y, 1) yield _wait(y2) assert y2.key in a.data or y2.key in b.data
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) yield _wait([x]) # learn that slowinc is slow x = c.submit(np.arange, 1000000, workers=a.address) # put expensive data yield _wait([x]) slow = [c.submit(slowinc, x, delay=0.1, pure=False) for i in range(20)] yield _wait([slow]) assert b.data # not empty
def test_dont_steal_fast_tasks(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 yield _wait(c.submit(do_nothing, 1)) futures = c.map(do_nothing, range(1000), y=x) yield _wait(futures) assert len(s.has_what[workers[0].address]) == 1001
def test_work_steal_no_kwargs(c, s, a, b): yield _wait(c.submit(slowinc, 1, delay=0.05)) futures = c.map(slowinc, range(100), workers=a.address, allow_other_workers=True, delay=0.05) yield _wait(futures) assert 20 < len(a.data) < 80 assert 20 < len(b.data) < 80 total = c.submit(sum, futures) result = yield total._result() assert result == sum(map(inc, range(100)))
def test_with_status(e, s, a, b): ss = HTTPScheduler(s) ss.listen(0) client = AsyncHTTPClient() response = yield client.fetch('http://localhost:%d/tasks.json' % ss.port) out = json.loads(response.body.decode()) assert out['total'] == 0 assert out['processing'] == 0 assert out['failed'] == 0 assert out['in-memory'] == 0 assert out['ready'] == 0 assert out['waiting'] == 0 L = e.map(div, range(10), range(10)) yield _wait(L) client = AsyncHTTPClient() response = yield client.fetch('http://localhost:%d/tasks.json' % ss.port) out = json.loads(response.body.decode()) assert out['failed'] == 1 assert out['in-memory'] == 9 assert out['ready'] == 0 assert out['total'] == 10 assert out['waiting'] == 0 ss.stop()
def test_balance_with_restrictions(client, s, a, b, c): [x], [y] = yield [client._scatter([[1, 2, 3]], workers=a.address), client._scatter([1], workers=c.address)] z = client.submit(inc, 1, workers=[a.address, c.address]) yield _wait(z) assert s.who_has[z.key] == {c.address}
def test_task_stream_clear_interval(c, s, a, b): ts = TaskStream(s, clear_interval=100) yield _wait(c.map(inc, range(10))) ts.update() yield gen.sleep(0.010) yield _wait(c.map(dec, range(10))) ts.update() assert len(ts.source.data['start']) == 20 yield gen.sleep(0.150) yield _wait(c.map(inc, range(10, 20))) ts.update() assert len(ts.source.data['start']) == 10
def test_task_stream_n_rectangles(c, s, a, b): ts = TaskStream(s, n_rectangles=10) futures = c.map(slowinc, range(10), delay=0.001) yield _wait(futures) ts.update() assert len(ts.source.data['start']) == 10
def test_move(c, s, a, b): [x] = yield c._scatter([1], workers=b.address) future = c.submit(inc, x, resources={'A': 1}) yield _wait(future) assert a.data[future.key] == 2
def test_balance_resources(c, s, a, b): futures = c.map(slowinc, range(100), delay=0.1, workers=a.address) constrained = c.map(inc, range(2), resources={'A': 1}) yield _wait(constrained) assert any(f.key in a.data for f in constrained) # share assert any(f.key in b.data for f in constrained)
def test_worker_who_has_clears_after_failed_connection(c, s, a, b): n = Nanny(s.ip, s.port, ncores=2, loop=s.loop) n.start(0) start = time() while len(s.ncores) < 3: yield gen.sleep(0.01) assert time() < start + 5 futures = c.map(slowinc, range(20), delay=0.01) yield _wait(futures) result = yield c.submit(sum, futures, workers=a.address) for dep in set(a.dep_state) - set(a.task_state): a.release_dep(dep, report=True) n_worker_address = n.worker_address with ignoring(CommClosedError): yield c._run(os._exit, 1, workers=[n_worker_address]) while len(s.workers) > 2: yield gen.sleep(0.01) total = c.submit(sum, futures, workers=a.address) yield total assert not a.has_what.get(n_worker_address) assert not any(n_worker_address in s for s in a.who_has.values()) yield n._close()
def test_with_data(e, s, a, b): ss = HTTPScheduler(s) ss.listen(0) L = e.map(inc, [1, 2, 3]) L2 = yield e._scatter(['Hello', 'world!']) yield _wait(L) client = AsyncHTTPClient() response = yield client.fetch('http://localhost:%d/memory-load.json' % ss.port) out = json.loads(response.body.decode()) assert all(isinstance(v, int) for v in out.values()) assert set(out) == {a.address, b.address} assert sum(out.values()) == sum(map(getsizeof, [1, 2, 3, 'Hello', 'world!'])) response = yield client.fetch('http://localhost:%s/memory-load-by-key.json' % ss.port) out = json.loads(response.body.decode()) assert set(out) == {a.address, b.address} assert all(isinstance(v, dict) for v in out.values()) assert all(k in {'inc', 'data'} for d in out.values() for k in d) assert all(isinstance(v, int) for d in out.values() for v in d.values()) assert sum(v for d in out.values() for v in d.values()) == \ sum(map(getsizeof, [1, 2, 3, 'Hello', 'world!'])) ss.stop()
def test_fail_write_many_to_disk(c, s, a, b): a.validate = False b.validate = False class Bad(object): def __init__(self, x): pass def __getstate__(self): raise TypeError() def __sizeof__(self): return 500 futures = c.map(Bad, range(10)) future = c.submit(lambda *args: 123, *futures) yield _wait(future) with pytest.raises(Exception) as info: yield future # workers still operational result = yield c.submit(inc, 1, workers=a.address) assert result == 2 result = yield c.submit(inc, 2, workers=b.address) assert result == 3
def test_share_communication(c, s, w1, w2, w3): x = c.submit(mul, b'1', int(w3.target_message_size + 1), workers=w1.address) y = c.submit(mul, b'2', int(w3.target_message_size + 1), workers=w2.address) yield _wait([x, y]) yield c._replicate([x, y], workers=[w1.address, w2.address]) z = c.submit(add, x, y, workers=w3.address) yield _wait(z) assert len(w3.incoming_transfer_log) == 2 assert w1.outgoing_transfer_log assert w2.outgoing_transfer_log
def test_with_data(e, s, a, b): ss = HTTPScheduler(s) ss.listen(0) L = e.map(inc, [1, 2, 3]) L2 = yield e._scatter(['Hello', 'world!']) yield _wait(L) client = AsyncHTTPClient() response = yield client.fetch('http://localhost:%d/memory-load.json' % ss.port) out = json.loads(response.body.decode()) assert all(isinstance(v, int) for v in out.values()) assert set(out) == {a.address, b.address} assert sum(out.values()) == sum( map(getsizeof, [1, 2, 3, 'Hello', 'world!'])) response = yield client.fetch( 'http://localhost:%s/memory-load-by-key.json' % ss.port) out = json.loads(response.body.decode()) assert set(out) == {a.address, b.address} assert all(isinstance(v, dict) for v in out.values()) assert all(k in {'inc', 'data'} for d in out.values() for k in d) assert all(isinstance(v, int) for d in out.values() for v in d.values()) assert sum(v for d in out.values() for v in d.values()) == \ sum(map(getsizeof, [1, 2, 3, 'Hello', 'world!'])) ss.stop()
def test_with_status(e, s, a, b): ss = HTTPScheduler(s) ss.listen(0) client = AsyncHTTPClient() response = yield client.fetch('http://localhost:%d/tasks.json' % ss.port) out = json.loads(response.body.decode()) assert out['total'] == 0 assert out['processing'] == 0 assert out['failed'] == 0 assert out['in-memory'] == 0 assert out['waiting'] == 0 L = e.map(div, range(10), range(10)) yield _wait(L) client = AsyncHTTPClient() response = yield client.fetch('http://localhost:%d/tasks.json' % ss.port) out = json.loads(response.body.decode()) assert out['failed'] == 1 assert out['in-memory'] == 9 assert out['total'] == 10 assert out['waiting'] == 0 ss.stop()
def test_dont_steal_long_running_tasks(c, s, a, b): def long(delay): with worker_client() as c: sleep(delay) yield c.submit(long, 0.1)._result() # learn duration yield c.submit(inc, 1)._result() # 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 yield gen.sleep(0.01) start = time() while any(t.key in s.extensions['stealing'].key_stealable for t in long_tasks): yield gen.sleep(0.01) assert time() < start + 1 na = len(a.executing) nb = len(b.executing) incs = c.map(inc, range(100), workers=a.address, allow_other_workers=True) yield gen.sleep(0.2) assert sum(1 for k in s.processing[b.address] if k.startswith('long')) <= nb yield _wait(long_tasks)
def test_write_bytes(e, s, a, b): with make_hdfs() as hdfs: data = [b'123', b'456', b'789'] remote_data = yield e._scatter(data) futures = write_bytes('/tmp/test/data/file.*.dat', remote_data, hdfs=hdfs) yield _wait(futures) assert len(hdfs.ls('/tmp/test/data/')) == 3 with hdfs.open('/tmp/test/data/file.1.dat') as f: assert f.read() == b'456' futures = write_bytes('/tmp/test/data2/', remote_data, hdfs=hdfs) yield _wait(futures) assert len(hdfs.ls('/tmp/test/data2/')) == 3
def test_workers(c, s, a, b): d = workers(s) assert json.loads(json.dumps(d)) == d assert 0 <= d[a.ip]['cpu'] <= 100 assert 0 <= d[a.ip]['memory'] assert 0 < d[a.ip]['memory_percent'] < 100 assert set(map(int, d[a.ip]['ports'])) == {a.port, b.port} assert d[a.ip]['processing'] == {} # assert d[a.ip]['last-seen'] > 0 L = c.map(div, range(10), range(10)) yield _wait(L) assert 0 <= d[a.ip]['cpu'] <= 100 assert 0 <= d[a.ip]['memory'] assert 0 < d[a.ip]['memory_percent'] < 100 assert set(map(int, d[a.ip]['ports'])) == {a.port, b.port} assert d[a.ip]['processing'] == {} try: assert 0 <= d[a.ip]['disk-read'] assert 0 <= d[a.ip]['disk-write'] except KeyError: import psutil with pytest.raises(RuntimeError): psutil.disk_io_counters() assert 0 <= d[a.ip]['network-send'] assert 0 <= d[a.ip]['network-recv']
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) yield _wait(futures) assert len(a.data) == 4 assert len(b.data) == 0
def test_work_stealing(c, s, a, b): [x] = yield c._scatter([1]) futures = c.map(slowadd, range(50), [x] * 50) yield gen.sleep(0.1) yield _wait(futures) assert len(a.data) > 10 assert len(b.data) > 10
def test_dont_work_steal(c, s, a, b): [x] = yield c._scatter([1], workers=a.address) futures = [c.submit(slowadd, x, i, resources={'A': 1}, delay=0.05) for i in range(10)] yield _wait(futures) assert all(f.key in a.data for f in futures)
def test_compute(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) yy = c.compute(y, resources={x: {'A': 1}, y: {'B': 1}}) yield _wait(yy) assert b.data
def test_submit_after_failed_worker(c, s, a, b): L = c.map(inc, range(10)) yield _wait(L) yield a._close() total = c.submit(sum, L) result = yield total assert result == sum(map(inc, range(10)))
def test_fifo_submission(c, s, w): futures = [] for i in range(20): future = c.submit(slowinc, i, delay=0.1, key='inc-%02d' % i) futures.append(future) yield gen.sleep(0.01) yield _wait(futures[-1]) assert futures[10].status == 'finished'
def test_worker_breaks_and_returns(c, s, a): future = c.submit(slowinc, 1, delay=0.1) for i in range(10): future = c.submit(slowinc, future, delay=0.1) yield _wait(future) a.batched_stream.comm.close() yield gen.sleep(0.1) start = time() yield _wait(future) end = time() assert end - start < 1 assert frequencies(s.task_state.values()) == {'memory': 1, 'released': 10}
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) yield _wait([x]) future = c.submit(np.sum, [1], workers=a.address) # learn that sum is fast yield _wait([future]) cheap = [ c.submit(np.sum, x, pure=False, workers=a.address, allow_other_workers=True) for i in range(10) ] yield _wait(cheap) assert len(b.data) == 0 assert len(a.data) == 12
def test_work_stealing(c, s, a, b): [x] = yield c._scatter([1], workers=a.address) futures = c.map(slowadd, range(50), [x] * 50) yield gen.sleep(0.1) yield _wait(futures) assert len(a.data) > 10 assert len(b.data) > 10 assert len(a.data) > len(b.data)
def test_eventstream(c, s, a, b): es = EventStream() s.add_plugin(es) assert es.buffer == [] futures = c.map(div, [1] * 10, range(10)) yield _wait(futures) assert len(es.buffer) == 10
def test_minimum_resource(c, s, a): futures = c.map(slowinc, range(30), resources={'A': 1, 'B': 1}, delay=0.02) while len(a.data) < 30: yield gen.sleep(0.01) assert len(a.executing) <= 1 yield _wait(futures) assert a.total_resources == a.available_resources
def test_learn_occupancy_multiple_workers(c, s, a, b): x = c.submit(slowinc, 1, delay=0.2, workers=a.address) yield gen.sleep(0.05) futures = c.map(slowinc, range(100), delay=0.2) yield _wait(x) assert not any(v == 0.5 for vv in s.processing.values() for v in vv) s.validate_state()
def test_task_stream_second_plugin(c, s, a, b): ts = TaskStream(s, n_rectangles=10, clear_interval=10) ts.update() futures = c.map(inc, range(10)) yield _wait(futures) ts.update() ts2 = TaskStream(s, n_rectangles=5, clear_interval=10) ts2.update()
def test_prefer_constrained(c, s, a): futures = c.map(slowinc, range(1000), delay=0.1) constrained = c.map(inc, range(10), resources={'A': 1}) start = time() yield _wait(constrained) end = time() assert end - start < 1 assert s.processing[a.address]
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) yield _wait(futures) assert len(a.data) >= 3 assert len(b.data) >= 3
def test_steal_twice(c, s, a, b): x = c.submit(inc, 1, workers=a.address) yield _wait(x) futures = [c.submit(slowadd, x, i, delay=0.2) for i in range(100)] while len(s.task_state) < 100: # tasks are all allocated yield gen.sleep(0.01) workers = [Worker(s.ip, s.port, loop=s.loop) for _ in range(30)] yield [w._start() for w in workers] # army of new workers arrives to help yield _wait(futures) assert all(s.has_what.values()) assert max(map(len, s.has_what.values())) < 20 yield [w._close() for w in workers]
def test_types(c, s, a, b): assert not a.types assert not b.types x = c.submit(inc, 1, workers=a.address) yield _wait(x) assert a.types[x.key] == int y = c.submit(inc, x, workers=b.address) yield _wait(y) assert b.types == {x.key: int, y.key: int} yield c._cancel(y) start = time() while y.key in b.data: yield gen.sleep(0.01) assert time() < start + 5 assert y.key not in b.types
def test_persist(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) xx, yy = c.persist([x, y], resources={x: {'A': 1}, y: {'B': 1}}) yield _wait([xx, yy]) assert x.key in a.data assert y.key in b.data
def test_worksteal_many_thieves(c, s, *workers): x = c.submit(slowinc, -1, delay=0.1) yield x._result() xs = c.map(slowinc, [x] * 100, pure=False, delay=0.01) yield _wait(xs) for w, keys in s.has_what.items(): assert 2 < len(keys) < 50
def test_multiple_transfers(c, s, w1, w2, w3): x = c.submit(inc, 1, workers=w1.address) y = c.submit(inc, 2, workers=w2.address) z = c.submit(add, x, y, workers=w3.address) yield _wait(z) r = w3.startstops[z.key] transfers = [t for t in r if t[0] == 'transfer'] assert len(transfers) == 2
def test_steal_when_more_tasks(c, s, a, *rest): s.extensions['stealing']._pc.callback_time = 20 x = c.submit(mul, b'0', 100000000, workers=a.address) # 100 MB yield _wait(x) s.task_duration['slowidentity'] = 0.2 futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(20)] yield gen.sleep(0.1) assert any(w.task_state for w in rest)
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)) yield _wait(futures) durations = [sum(w.data.values()) for w in workers] assert max(durations) / min(durations) < 2
def test_persist_tuple(c, s, a, b): x = delayed(inc)(1) y = delayed(inc)(x) xx, yy = c.persist([x, y], resources={(x, y): {'A': 1}}) yield _wait([xx, yy]) assert x.key in a.data assert y.key in a.data assert not b.data
def test_submit_many_non_overlapping(c, s, a, b): futures = c.map(slowinc, range(100), resources={'A': 1}, delay=0.02) while len(a.data) + len(b.data) < 100: yield gen.sleep(0.01) assert len(a.executing) <= 2 assert len(b.executing) <= 1 yield _wait(futures) assert a.total_resources == a.available_resources assert b.total_resources == b.available_resources
def test_resource_submit(c, s, a, b): x = c.submit(inc, 1, resources={'A': 3}) y = c.submit(inc, 2, resources={'B': 1}) z = c.submit(inc, 3, resources={'C': 2}) yield _wait(x) assert x.key in a.data yield _wait(y) assert y.key in b.data assert z.key in s.unrunnable d = Worker(s.ip, s.port, loop=s.loop, resources={'C': 10}) yield d._start() yield _wait(z) assert z.key in d.data yield d._close()
def test_active_holds_tasks(e, s, w): future = e.submit(slowinc, 1, delay=0.2) yield gen.sleep(0.1) assert future.key in w.active yield future._result() assert future.key not in w.active future = e.submit(throws, 1) with ignoring(Exception): yield _wait([future]) assert not w.active
def test_cancel_stress(c, s, *workers): da = pytest.importorskip('dask.array') x = da.random.random((40, 40), chunks=(1, 1)) x = c.persist(x) yield _wait([x]) y = (x.sum(axis=0) + x.sum(axis=1) + 1).std() for i in range(5): f = c.compute(y) while len(s.waiting) > (len(y.dask) - len(x.dask)) / 2: yield gen.sleep(0.01) yield c._cancel(f)