From ee1008416eb73418c6b656a1ab62af62c2087ffc Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Sun, 12 May 2019 10:03:36 -0500 Subject: [PATCH 01/43] Organize thread/process/instance checking in utils_test.py (#2687) This collects various state checking functionality in one place. It also makes some modifications that were previously causing Dask tests to fail. --- distributed/utils_test.py | 546 +++++++++++++++++++------------------- 1 file changed, 280 insertions(+), 266 deletions(-) diff --git a/distributed/utils_test.py b/distributed/utils_test.py index 4cef981cd0..e40912fcb3 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -148,15 +148,7 @@ def start(): is_stopped.wait() Worker._instances.clear() - start = time() - while set(_global_clients): - sleep(0.1) - assert time() < start + 10 - _cleanup_dangling() - - assert_no_leaked_processes() - _global_clients.clear() @@ -630,158 +622,123 @@ def cluster( nworkers=2, nanny=False, worker_kwargs={}, active_rpc_timeout=1, scheduler_kwargs={} ): ws = weakref.WeakSet() - - reset_config() - Comm._instances.clear() - - for name, level in logging_levels.items(): - logging.getLogger(name).setLevel(level) - enable_proctitle_on_children() - with pristine_loop() as loop: - with check_active_rpc(loop, active_rpc_timeout): - if nanny: - _run_worker = run_nanny - else: - _run_worker = run_worker + with clean(timeout=active_rpc_timeout, threads=False) as loop: + if nanny: + _run_worker = run_nanny + else: + _run_worker = run_worker - # The scheduler queue will receive the scheduler's address - scheduler_q = mp_context.Queue() + # The scheduler queue will receive the scheduler's address + scheduler_q = mp_context.Queue() - # Launch scheduler - scheduler = mp_context.Process( - name="Dask cluster test: Scheduler", - target=run_scheduler, - args=(scheduler_q, nworkers + 1), - kwargs=scheduler_kwargs, + # Launch scheduler + scheduler = mp_context.Process( + name="Dask cluster test: Scheduler", + target=run_scheduler, + args=(scheduler_q, nworkers + 1), + kwargs=scheduler_kwargs, + ) + ws.add(scheduler) + scheduler.daemon = True + scheduler.start() + + # Launch workers + workers = [] + for i in range(nworkers): + q = mp_context.Queue() + fn = "_test_worker-%s" % uuid.uuid4() + kwargs = merge( + {"ncores": 1, "local_dir": fn, "memory_limit": TOTAL_MEMORY}, + worker_kwargs, ) - ws.add(scheduler) - scheduler.daemon = True - scheduler.start() - - # Launch workers - workers = [] - for i in range(nworkers): - q = mp_context.Queue() - fn = "_test_worker-%s" % uuid.uuid4() - kwargs = merge( - {"ncores": 1, "local_dir": fn, "memory_limit": TOTAL_MEMORY}, - worker_kwargs, - ) - proc = mp_context.Process( - name="Dask cluster test: Worker", - target=_run_worker, - args=(q, scheduler_q), - kwargs=kwargs, - ) - ws.add(proc) - workers.append({"proc": proc, "queue": q, "dir": fn}) + proc = mp_context.Process( + name="Dask cluster test: Worker", + target=_run_worker, + args=(q, scheduler_q), + kwargs=kwargs, + ) + ws.add(proc) + workers.append({"proc": proc, "queue": q, "dir": fn}) + for worker in workers: + worker["proc"].start() + try: for worker in workers: - worker["proc"].start() - try: - for worker in workers: - worker["address"] = worker["queue"].get(timeout=5) - except Empty: - raise pytest.xfail.Exception("Worker failed to start in test") + worker["address"] = worker["queue"].get(timeout=5) + except Empty: + raise pytest.xfail.Exception("Worker failed to start in test") - saddr = scheduler_q.get() + saddr = scheduler_q.get() - start = time() + start = time() + try: try: - try: - security = scheduler_kwargs["security"] - rpc_kwargs = { - "connection_args": security.get_connection_args("client") - } - except KeyError: - rpc_kwargs = {} - - with rpc(saddr, **rpc_kwargs) as s: - while True: - ncores = loop.run_sync(s.ncores) - if len(ncores) == nworkers: - break - if time() - start > 5: - raise Exception("Timeout on cluster creation") - - # avoid sending processes down to function - yield {"address": saddr}, [ - {"address": w["address"], "proc": weakref.ref(w["proc"])} - for w in workers - ] - finally: - logger.debug("Closing out test cluster") - - loop.run_sync( - lambda: disconnect_all( - [w["address"] for w in workers], - timeout=0.5, - rpc_kwargs=rpc_kwargs, - ) - ) - loop.run_sync( - lambda: disconnect(saddr, timeout=0.5, rpc_kwargs=rpc_kwargs) + security = scheduler_kwargs["security"] + rpc_kwargs = {"connection_args": security.get_connection_args("client")} + except KeyError: + rpc_kwargs = {} + + with rpc(saddr, **rpc_kwargs) as s: + while True: + ncores = loop.run_sync(s.ncores) + if len(ncores) == nworkers: + break + if time() - start > 5: + raise Exception("Timeout on cluster creation") + + # avoid sending processes down to function + yield {"address": saddr}, [ + {"address": w["address"], "proc": weakref.ref(w["proc"])} + for w in workers + ] + finally: + logger.debug("Closing out test cluster") + + loop.run_sync( + lambda: disconnect_all( + [w["address"] for w in workers], timeout=0.5, rpc_kwargs=rpc_kwargs ) + ) + loop.run_sync(lambda: disconnect(saddr, timeout=0.5, rpc_kwargs=rpc_kwargs)) - scheduler.terminate() - scheduler_q.close() - scheduler_q._reader.close() - scheduler_q._writer.close() + scheduler.terminate() + scheduler_q.close() + scheduler_q._reader.close() + scheduler_q._writer.close() - for w in workers: - w["proc"].terminate() - w["queue"].close() - w["queue"]._reader.close() - w["queue"]._writer.close() + for w in workers: + w["proc"].terminate() + w["queue"].close() + w["queue"]._reader.close() + w["queue"]._writer.close() - scheduler.join(2) - del scheduler - for proc in [w["proc"] for w in workers]: - proc.join(timeout=2) + scheduler.join(2) + del scheduler + for proc in [w["proc"] for w in workers]: + proc.join(timeout=2) - with ignoring(UnboundLocalError): - del worker, w, proc - del workers[:] + with ignoring(UnboundLocalError): + del worker, w, proc + del workers[:] - for fn in glob("_test_worker-*"): - with ignoring(OSError): - shutil.rmtree(fn) + for fn in glob("_test_worker-*"): + with ignoring(OSError): + shutil.rmtree(fn) - try: - client = default_client() - except ValueError: - pass - else: - client.close() + try: + client = default_client() + except ValueError: + pass + else: + client.close() start = time() while list(ws): sleep(0.01) assert time() < start + 1, "Workers still around after one second" - for i in range(5): - if all(c.closed() for c in Comm._instances): - break - else: - sleep(0.1) - else: - L = [c for c in Comm._instances if not c.closed()] - Comm._instances.clear() - print("Unclosed Comms", L) - # raise ValueError("Unclosed Comms", L) - - assert_no_leaked_processes() - - -def assert_no_leaked_processes(): - for i in range(20): - if mp_context.active_children(): - sleep(0.1) - else: - assert not mp_context.active_children() - @gen.coroutine def disconnect(addr, timeout=3, rpc_kwargs=None): @@ -922,147 +879,95 @@ def _(func): func = gen.coroutine(func) def test_func(): - Client._instances.clear() - Worker._instances.clear() - Scheduler._instances.clear() - Nanny._instances.clear() - _global_clients.clear() - Comm._instances.clear() - active_threads_start = set(threading._active) - - reset_config() - - dask.config.set({"distributed.comm.timeouts.connect": "5s"}) - # Restore default logging levels - # XXX use pytest hooks/fixtures instead? - for name, level in logging_levels.items(): - logging.getLogger(name).setLevel(level) - result = None workers = [] + with clean(threads=check_new_threads, timeout=active_rpc_timeout) as loop: - with pristine_loop() as loop: - with check_active_rpc(loop, active_rpc_timeout): - - @gen.coroutine - def coro(): - with dask.config.set(config): - s = False - for i in range(5): - try: - s, ws = yield start_cluster( - ncores, - scheduler, - loop, - security=security, - Worker=Worker, - scheduler_kwargs=scheduler_kwargs, - worker_kwargs=worker_kwargs, - ) - except Exception as e: - logger.error( - "Failed to start gen_cluster, retrying", - exc_info=True, - ) - else: - workers[:] = ws - args = [s] + workers - break - if s is False: - raise Exception("Could not start cluster") - if client: - c = yield Client( - s.address, - loop=loop, + @gen.coroutine + def coro(): + with dask.config.set(config): + s = False + for i in range(5): + try: + s, ws = yield start_cluster( + ncores, + scheduler, + loop, security=security, - asynchronous=True, - **client_kwargs + Worker=Worker, + scheduler_kwargs=scheduler_kwargs, + worker_kwargs=worker_kwargs, ) - args = [c] + args - try: - future = func(*args) - if timeout: - future = gen.with_timeout( - timedelta(seconds=timeout), future - ) - result = yield future - if s.validate: - s.validate_state() - finally: - if client and c.status not in ("closing", "closed"): - yield c._close(fast=s.status == "closed") - yield end_cluster(s, workers) - yield gen.with_timeout( - timedelta(seconds=1), cleanup_global_workers() + except Exception as e: + logger.error( + "Failed to start gen_cluster, retrying", + exc_info=True, ) - - try: - c = yield default_client() - except ValueError: - pass - else: - yield c._close(fast=True) - - for i in range(5): - if all(c.closed() for c in Comm._instances): - break - else: - yield gen.sleep(0.05) else: - L = [c for c in Comm._instances if not c.closed()] - Comm._instances.clear() - # raise ValueError("Unclosed Comms", L) - print("Unclosed Comms", L) - - raise gen.Return(result) - - result = loop.run_sync( - coro, timeout=timeout * 2 if timeout else timeout - ) + workers[:] = ws + args = [s] + workers + break + if s is False: + raise Exception("Could not start cluster") + if client: + c = yield Client( + s.address, + loop=loop, + security=security, + asynchronous=True, + **client_kwargs + ) + args = [c] + args + try: + future = func(*args) + if timeout: + future = gen.with_timeout( + timedelta(seconds=timeout), future + ) + result = yield future + if s.validate: + s.validate_state() + finally: + if client and c.status not in ("closing", "closed"): + yield c._close(fast=s.status == "closed") + yield end_cluster(s, workers) + yield gen.with_timeout( + timedelta(seconds=1), cleanup_global_workers() + ) - for w in workers: - if getattr(w, "data", None): try: - w.data.clear() - except EnvironmentError: - # zict backends can fail if their storage directory - # was already removed + c = yield default_client() + except ValueError: pass - del w.data - DequeHandler.clear_all_instances() - for w in Worker._instances: - w.close(report=False, executor_wait=False) - if w.status == "running": - w.close() - Worker._instances.clear() - - if PY3 and not WINDOWS and check_new_threads: - start = time() - while True: - bad = [ - t - for t, v in threading._active.items() - if t not in active_threads_start - and "Threaded" not in v.name - and "watch message" not in v.name - and "TCP-Executor" not in v.name - ] - if not bad: - break - else: - sleep(0.01) - if time() > start + 5: - from distributed import profile + else: + yield c._close(fast=True) - tid = bad[0] - thread = threading._active[tid] - call_stacks = profile.call_stack(sys._current_frames()[tid]) - assert False, (thread, call_stacks) - _cleanup_dangling() - with ignoring(AttributeError): - del thread_state.on_event_loop_thread + for i in range(5): + if all(c.closed() for c in Comm._instances): + break + else: + yield gen.sleep(0.05) + else: + L = [c for c in Comm._instances if not c.closed()] + Comm._instances.clear() + # raise ValueError("Unclosed Comms", L) + print("Unclosed Comms", L) - assert_no_leaked_processes() + raise gen.Return(result) + + result = loop.run_sync( + coro, timeout=timeout * 2 if timeout else timeout + ) + + for w in workers: + if getattr(w, "data", None): + try: + w.data.clear() + except EnvironmentError: + # zict backends can fail if their storage directory + # was already removed + pass + del w.data return result @@ -1510,3 +1415,112 @@ def gen_tls_cluster(**kwargs): return gen_cluster( scheduler="tls://127.0.0.1", security=tls_only_security(), **kwargs ) + + +@contextmanager +def check_thread_leak(): + active_threads_start = set(threading._active) + + yield + + start = time() + while True: + bad = [ + t + for t, v in threading._active.items() + if t not in active_threads_start + and "Threaded" not in v.name + and "watch message" not in v.name + and "TCP-Executor" not in v.name + ] + if not bad: + break + else: + sleep(0.01) + if time() > start + 5: + from distributed import profile + + tid = bad[0] + thread = threading._active[tid] + call_stacks = profile.call_stack(sys._current_frames()[tid]) + assert False, (thread, call_stacks) + + +@contextmanager +def check_process_leak(): + start_children = set(mp_context.active_children()) + + yield + + for i in range(50): + if not set(mp_context.active_children()) - start_children: + break + else: + sleep(0.2) + else: + assert not mp_context.active_children() + + _cleanup_dangling() + + +@contextmanager +def check_instances(): + Client._instances.clear() + Worker._instances.clear() + Scheduler._instances.clear() + Nanny._instances.clear() + _global_clients.clear() + Comm._instances.clear() + + yield + + start = time() + while set(_global_clients): + sleep(0.1) + assert time() < start + 10 + + _global_clients.clear() + + for w in Worker._instances: + w.close(report=False, executor_wait=False) + if w.status == "running": + w.close() + Worker._instances.clear() + + for i in range(5): + if all(c.closed() for c in Comm._instances): + break + else: + sleep(0.1) + else: + L = [c for c in Comm._instances if not c.closed()] + Comm._instances.clear() + print("Unclosed Comms", L) + # raise ValueError("Unclosed Comms", L) + + DequeHandler.clear_all_instances() + + +@contextmanager +def clean(threads=not WINDOWS, processes=True, instances=True, timeout=1): + @contextmanager + def null(): + yield + + with check_thread_leak() if threads else null(): + with pristine_loop() as loop: + with check_process_leak() if processes else null(): + with check_instances() if instances else null(): + with check_active_rpc(loop, timeout): + reset_config() + + dask.config.set({"distributed.comm.timeouts.connect": "5s"}) + # Restore default logging levels + # XXX use pytest hooks/fixtures instead? + for name, level in logging_levels.items(): + logging.getLogger(name).setLevel(level) + + yield loop + + with ignoring(AttributeError): + del thread_state.on_event_loop_thread From 3142dda225baaf1db0b8c73f240f7fad8941fc1e Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 13 May 2019 08:11:20 -0500 Subject: [PATCH 02/43] Learn bandwidth over time (#2658) In order to schedule tasks intelligently we need to know how long communications will take. To do this, we need to estimate the bandwidth of the network. This can vary by orders of magnitude depending on hardwware. Previously we asked the user to specify this in configuration. Now we learn it over time. Each worker keeps an exponentially weighted moving average for all of its data communications. It sends this information to the scheduler as part of the heartbeats (which include lots of other diagnostic information). The scheduler updates its own measurement accordingly. --- distributed/scheduler.py | 9 ++++++--- distributed/stealing.py | 3 +-- distributed/tests/test_scheduler.py | 28 +++++++++++++++++++++------- distributed/tests/test_steal.py | 8 ++++---- distributed/utils.py | 2 ++ distributed/worker.py | 11 +++++++++-- 6 files changed, 43 insertions(+), 18 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 68a80ac664..446b479c76 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -54,6 +54,7 @@ no_default, DequeHandler, parse_timedelta, + parse_bytes, PeriodicCallback, shutting_down, ) @@ -72,7 +73,6 @@ logger = logging.getLogger(__name__) -BANDWIDTH = dask.config.get("distributed.scheduler.bandwidth") ALLOWED_FAILURES = dask.config.get("distributed.scheduler.allowed-failures") LOG_PDB = dask.config.get("distributed.admin.pdb-on-err") @@ -868,6 +868,7 @@ def __init__( else: self.idle_timeout = None self.time_started = time() + self.bandwidth = parse_bytes(dask.config.get("distributed.scheduler.bandwidth")) self.security = security or Security() assert isinstance(self.security, Security) @@ -1359,6 +1360,8 @@ def heartbeat_worker( host_info = host_info or {} self.host_info[host]["last-seen"] = local_now + frac = 1 / 20 / len(self.workers) + self.bandwidth = self.bandwidth * (1 - frac) + metrics["bandwidth"] * frac ws = self.workers.get(address) if not ws: @@ -3336,7 +3339,7 @@ def get_comm_cost(self, ts, ws): Get the estimated communication cost (in s.) to compute the task on the given worker. """ - return sum(dts.nbytes for dts in ts.dependencies - ws.has_what) / BANDWIDTH + return sum(dts.nbytes for dts in ts.dependencies - ws.has_what) / self.bandwidth def get_task_duration(self, ts, default=0.5): """ @@ -4522,7 +4525,7 @@ def worker_objective(self, ts, ws): [dts.get_nbytes() for dts in ts.dependencies if ws not in dts.who_has] ) stack_time = ws.occupancy / ws.ncores - start_time = comm_bytes / BANDWIDTH + stack_time + start_time = comm_bytes / self.bandwidth + stack_time if ts.actor: return (len(ws.actors), start_time, ws.nbytes) diff --git a/distributed/stealing.py b/distributed/stealing.py index d361305b10..dc8c989e39 100644 --- a/distributed/stealing.py +++ b/distributed/stealing.py @@ -15,7 +15,6 @@ except ImportError: from toolz import topk -BANDWIDTH = 100e6 LATENCY = 10e-3 log_2 = log(2) @@ -134,7 +133,7 @@ def steal_time_ratio(self, ts): nbytes = sum(dep.get_nbytes() for dep in ts.dependencies) - transfer_time = nbytes / BANDWIDTH + LATENCY + transfer_time = nbytes / self.scheduler.bandwidth + LATENCY split = ts.prefix if split in fast_tasks: return None, None diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index ceb992d6e2..9f61e5e710 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -5,7 +5,7 @@ from collections import defaultdict from datetime import timedelta import json -from operator import add, mul +import operator import sys from time import sleep @@ -18,7 +18,7 @@ from distributed import Nanny, Worker, Client, wait, fire_and_forget from distributed.core import connect, rpc -from distributed.scheduler import Scheduler, BANDWIDTH +from distributed.scheduler import Scheduler from distributed.client import wait from distributed.metrics import time from distributed.protocol.pickle import dumps @@ -63,7 +63,7 @@ def test_respect_data_in_memory(c, s, a): assert s.tasks[y.key].who_has == {s.workers[a.address]} - z = delayed(add)(x, y) + z = delayed(operator.add)(x, y) f2 = c.persist(z) while f2.key not in s.tasks or not s.tasks[f2.key]: assert s.tasks[y.key].who_has @@ -427,7 +427,10 @@ def test_filtered_communication(s, a, b): yield f.write( { "op": "update-graph", - "tasks": {"x": dumps_task((inc, 1)), "z": dumps_task((add, "x", 10))}, + "tasks": { + "x": dumps_task((inc, 1)), + "z": dumps_task((operator.add, "x", 10)), + }, "dependencies": {"x": [], "z": ["x"]}, "client": "f", "keys": ["z"], @@ -903,8 +906,8 @@ def test_learn_occupancy_multiple_workers(c, s, a, b): @gen_cluster(client=True) def test_include_communication_in_occupancy(c, s, a, b): s.task_duration["slowadd"] = 0.001 - x = c.submit(mul, b"0", int(BANDWIDTH), workers=a.address) - y = c.submit(mul, b"1", int(BANDWIDTH * 1.5), workers=b.address) + x = c.submit(operator.mul, b"0", int(s.bandwidth), workers=a.address) + y = c.submit(operator.mul, b"1", int(s.bandwidth * 1.5), workers=b.address) z = c.submit(slowadd, x, y, delay=1) while z.key not in s.tasks or not s.tasks[z.key].processing_on: @@ -1375,7 +1378,7 @@ def test_dont_recompute_if_persisted_3(c, s, a, b): x = delayed(inc)(1, dask_key_name="x") y = delayed(inc)(2, dask_key_name="y") z = delayed(inc)(y, dask_key_name="z") - w = delayed(add)(x, z, dask_key_name="w") + w = delayed(operator.add)(x, z, dask_key_name="w") ww = w.persist() yield wait(ww) @@ -1513,6 +1516,17 @@ def test_idle_timeout(c, s, a, b): assert b.status == "closed" +@gen_cluster(client=True, config={"distributed.scheduler.bandwidth": "100 GB"}) +def test_bandwidth(c, s, a, b): + start = s.bandwidth + x = c.submit(operator.mul, b"0", 20000, workers=a.address) + y = c.submit(lambda x: x, x, workers=b.address) + yield y + yield b.heartbeat() + assert s.bandwidth < start # we've learned that we're slower + assert b.latency + + @gen_cluster() def test_workerstate_clean(s, a, b): ws = s.workers[a.address].clean() diff --git a/distributed/tests/test_steal.py b/distributed/tests/test_steal.py index 8edeb8e339..7348d164c7 100644 --- a/distributed/tests/test_steal.py +++ b/distributed/tests/test_steal.py @@ -14,7 +14,7 @@ from distributed import Nanny, Worker, wait, worker_client from distributed.config import config from distributed.metrics import time -from distributed.scheduler import BANDWIDTH, key_split +from distributed.scheduler import key_split from distributed.utils_test import ( slowinc, slowadd, @@ -394,7 +394,7 @@ def assert_balanced(inp, expected, c, s, *workers): ts = s.tasks[dat.key] # Ensure scheduler state stays consistent old_nbytes = ts.nbytes - ts.nbytes = BANDWIDTH * t + ts.nbytes = s.bandwidth * t for ws in ts.who_has: ws.nbytes += ts.nbytes - old_nbytes else: @@ -499,8 +499,8 @@ def test_restart(c, s, a, b): def test_steal_communication_heavy_tasks(c, s, a, b): steal = s.extensions["stealing"] s.task_duration["slowadd"] = 0.001 - x = c.submit(mul, b"0", int(BANDWIDTH), workers=a.address) - y = c.submit(mul, b"1", int(BANDWIDTH), workers=b.address) + 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( diff --git a/distributed/utils.py b/distributed/utils.py index d6cc5ba62c..765035e5c1 100644 --- a/distributed/utils.py +++ b/distributed/utils.py @@ -1221,6 +1221,8 @@ def parse_bytes(s): >>> parse_bytes('MB') 1000000 """ + if isinstance(s, (int, float)): + return int(s) s = s.replace(" ", "") if not s[0].isdigit(): s = "1" + s diff --git a/distributed/worker.py b/distributed/worker.py index f5f4f2bdd5..8b8c139f35 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -400,6 +400,8 @@ def __init__( self.outgoing_count = 0 self.outgoing_current_count = 0 self.repetitively_busy = 0 + self.bandwidth = parse_bytes(dask.config.get("distributed.scheduler.bandwidth")) + self.latency = 0.001 self._client = None profile_cycle_interval = kwargs.pop( @@ -673,6 +675,7 @@ def get_metrics(self): in_memory=len(self.data), ready=len(self.ready), in_flight=len(self.in_flight_tasks), + bandwidth=self.bandwidth, ) custom = {k: metric(self) for k, metric in self.metrics.items()} @@ -742,6 +745,7 @@ def _register_with_scheduler(self): response = yield future _end = time() middle = (_start + _end) / 2 + self.latency = (_end - start) * 0.05 + self.latency * 0.95 self.scheduler_delay = response["time"] - middle self.status = "running" break @@ -1837,7 +1841,8 @@ def gather_dep(self, worker, dep, deps, total_nbytes, cause=None): ) total_bytes = sum(self.nbytes.get(dep, 0) for dep in response["data"]) - duration = (stop - start) or 0.5 + duration = (stop - start) or 0.010 + bandwidth = total_bytes / duration self.incoming_transfer_log.append( { "start": start + self.scheduler_delay, @@ -1848,10 +1853,12 @@ def gather_dep(self, worker, dep, deps, total_nbytes, cause=None): dep: self.nbytes.get(dep, None) for dep in response["data"] }, "total": total_bytes, - "bandwidth": total_bytes / duration, + "bandwidth": bandwidth, "who": worker, } ) + if total_bytes > 10000: + self.bandwidth = self.bandwidth * 0.95 + bandwidth * 0.05 if self.digests is not None: self.digests["transfer-bandwidth"].add(total_bytes / duration) self.digests["transfer-duration"].add(duration) From e3ffb9d12824b5a5434c5c15a2b6edfa7bf9ea12 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Thu, 9 May 2019 08:01:35 -0500 Subject: [PATCH 03/43] Use config accessor method for "scheduler-address" (#2676) --- distributed/cli/dask_worker.py | 8 ++++++-- distributed/cli/tests/test_dask_worker.py | 12 ++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/distributed/cli/dask_worker.py b/distributed/cli/dask_worker.py index a0bc801a96..1e0ebe2417 100755 --- a/distributed/cli/dask_worker.py +++ b/distributed/cli/dask_worker.py @@ -8,8 +8,8 @@ import warnings import click +import dask from distributed import Nanny, Worker -from distributed.config import config from distributed.utils import get_ip_interface, parse_timedelta from distributed.worker import _ncores from distributed.security import Security @@ -322,7 +322,11 @@ def del_pid_file(): kwargs["service_ports"] = {"nanny": nanny_port} t = Worker - if not scheduler and not scheduler_file and "scheduler-address" not in config: + if ( + not scheduler + and not scheduler_file + and dask.config.get("scheduler-address", None) is None + ): raise ValueError( "Need to provide scheduler address like\n" "dask-worker SCHEDULER_ADDRESS:8786" diff --git a/distributed/cli/tests/test_dask_worker.py b/distributed/cli/tests/test_dask_worker.py index 72084e5314..e1cfc8d5ad 100644 --- a/distributed/cli/tests/test_dask_worker.py +++ b/distributed/cli/tests/test_dask_worker.py @@ -6,6 +6,7 @@ import requests import sys +import os from time import sleep from toolz import first @@ -142,6 +143,17 @@ def test_scheduler_file(loop, nanny): assert time() < start + 10 +def test_scheduler_address_env(loop, monkeypatch): + monkeypatch.setenv("DASK_SCHEDULER_ADDRESS", "tcp://127.0.0.1:8786") + with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-worker", "--no-bokeh"]): + with Client(os.environ["DASK_SCHEDULER_ADDRESS"], loop=loop) as c: + start = time() + while not c.scheduler_info()["workers"]: + sleep(0.1) + assert time() < start + 10 + + def test_nprocs_requires_nanny(loop): with popen(["dask-scheduler", "--no-bokeh"]) as sched: with popen( From 61be3a78375b730b78ce4e50aecea2fe432cdddd Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 13 May 2019 16:48:31 -0500 Subject: [PATCH 04/43] bump version to 1.28.1 --- docs/source/changelog.rst | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 1b584c0221..613773a0c1 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,6 +1,14 @@ Changelog ========= +1.28.1 - 2019-05-13 +------------------- + +This is a small bugfix release due to a config change upstream. + +- Use config accessor method for "scheduler-address" (#2676) `James Bourbeau`_ + + 1.28.0 - 2019-05-08 ------------------- @@ -1030,4 +1038,4 @@ significantly without many new features. .. _`Peter Andreas Entschev`: https://github.com/pentschev .. _`condoratberlin`: https://github.com/condoratberlin .. _`K.-Michael Aye`: https://github.com/michaelaye -.. _`@plbertrand`: https://github.com/plbertrand \ No newline at end of file +.. _`@plbertrand`: https://github.com/plbertrand From a622ce4ac48861b07a2738baba361b9886ee1ca3 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 13 May 2019 18:13:48 -0500 Subject: [PATCH 05/43] Remove support for Iterators and Queues (#2671) These add non-trivial code complexity, and don't seem to be commonly used (based on bug reports and SO questions). They're also a bit odd on our tests (there are some lingering threads as a result. This commit removes functionality for them and replaces them with informative warnings pointing people towards normal for loops. --- distributed/client.py | 156 +++++------------------ distributed/tests/test_client.py | 206 +------------------------------ distributed/tests/test_utils.py | 23 +--- distributed/utils.py | 38 +----- distributed/utils_test.py | 2 +- docs/source/queues.rst | 196 +---------------------------- 6 files changed, 40 insertions(+), 581 deletions(-) diff --git a/distributed/client.py b/distributed/client.py index e31b808abc..7b6e14aa4e 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -55,7 +55,6 @@ from .cfexecutor import ClientExecutor from .compatibility import ( Queue as pyQueue, - Empty, isqueue, html_escape, StopAsyncIteration, @@ -77,7 +76,6 @@ sync, funcname, ignoring, - queue_to_iterator, tokey, log_errors, str_graph, @@ -1424,24 +1422,6 @@ def submit(self, func, *args, **kwargs): return futures[skey] - def _threaded_map(self, q_out, func, qs_in, **kwargs): - """ Internal function for mapping Queue """ - if isqueue(qs_in[0]): - get = pyQueue.get - elif isinstance(qs_in[0], Iterator): - get = next - else: - raise NotImplementedError() - - while True: - try: - args = [get(q) for q in qs_in] - except StopIteration as e: - q_out.put(e) - break - f = self.submit(func, *args, **kwargs) - q_out.put(f) - def map(self, func, *iterables, **kwargs): """ Map a function on a sequence of arguments @@ -1450,7 +1430,8 @@ def map(self, func, *iterables, **kwargs): Parameters ---------- func: callable - iterables: Iterables, Iterators, or Queues + iterables: Iterables + List-like objects to map over. They should have the same length. key: str, list Prefix for task names if string. Explicit names if list. pure: bool (defaults to True) @@ -1489,20 +1470,10 @@ def map(self, func, *iterables, **kwargs): if all(map(isqueue, iterables)) or all( isinstance(i, Iterator) for i in iterables ): - maxsize = kwargs.pop("maxsize", 0) - q_out = pyQueue(maxsize=maxsize) - t = threading.Thread( - target=self._threaded_map, - name="Threaded map()", - args=(q_out, func, iterables), - kwargs=kwargs, + raise TypeError( + "Dask no longer supports mapping over Iterators or Queues." + "Consider using a normal for loop and Client.submit" ) - t.daemon = True - t.start() - if isqueue(iterables[0]): - return q_out - else: - return queue_to_iterator(q_out) key = kwargs.pop("key", None) key = key or funcname(func) @@ -1738,22 +1709,7 @@ def _gather_remote(self, direct, local_worker): raise gen.Return(response) - def _threaded_gather(self, qin, qout, **kwargs): - """ Internal function for gathering Queue """ - while True: - L = [qin.get()] - while qin.empty(): - try: - L.append(qin.get_nowait()) - except Empty: - break - results = self.gather(L, **kwargs) - for item in results: - qout.put(item) - - def gather( - self, futures, errors="raise", maxsize=0, direct=None, asynchronous=None - ): + def gather(self, futures, errors="raise", direct=None, asynchronous=None): """ Gather futures from distributed memory Accepts a future, nested container of futures, iterator, or queue. @@ -1763,7 +1719,7 @@ def gather( ---------- futures: Collection of futures This can be a possibly nested collection of Future objects. - Collections can be lists, sets, iterators, queues or dictionaries + Collections can be lists, sets, or dictionaries errors: string Either 'raise' or 'skip' if we should raise if a future has erred or skip its inclusion in the output collection @@ -1771,9 +1727,6 @@ def gather( Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary. This can also be set when creating the Client. - maxsize: int - If the input is a queue then this produces an output queue with a - maximum size. Returns ------- @@ -1790,25 +1743,16 @@ def gather( >>> c.gather([x, [x], x]) # support lists and dicts # doctest: +SKIP [3, [3], 3] - >>> seq = c.gather(iter([x, x])) # support iterators # doctest: +SKIP - >>> next(seq) # doctest: +SKIP - 3 - See Also -------- Client.scatter: Send data out to cluster """ if isqueue(futures): - qout = pyQueue(maxsize=maxsize) - t = threading.Thread( - target=self._threaded_gather, - name="Threaded gather()", - args=(futures, qout), - kwargs={"errors": errors, "direct": direct}, + raise TypeError( + "Dask no longer supports gathering over Iterators and Queues. " + "Consider using a normal for loop and Client.submit/gather" ) - t.daemon = True - t.start() - return qout + elif isinstance(futures, Iterator): return (self.gather(f, errors=errors, direct=direct) for f in futures) else: @@ -1935,27 +1879,6 @@ def _scatter( out = list(out.values())[0] raise gen.Return(out) - def _threaded_scatter(self, q_or_i, qout, **kwargs): - """ Internal function for scattering Iterable/Queue data """ - while True: - if isqueue(q_or_i): - L = [q_or_i.get()] - while not q_or_i.empty(): - try: - L.append(q_or_i.get_nowait()) - except Empty: - break - else: - try: - L = [next(q_or_i)] - except StopIteration as e: - qout.put(e) - break - - futures = self.scatter(L, **kwargs) - for future in futures: - qout.put(future) - def scatter( self, data, @@ -1963,7 +1886,6 @@ def scatter( broadcast=False, direct=None, hash=True, - maxsize=0, timeout=no_default, asynchronous=None, ): @@ -1976,7 +1898,7 @@ def scatter( Parameters ---------- - data: list, iterator, dict, Queue, or object + data: list, dict, or object Data to scatter out to workers. Output type matches input type. workers: list of tuples (optional) Optionally constrain locations of data. @@ -1988,8 +1910,6 @@ def scatter( Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary. This can also be set when creating the Client. - maxsize: int (optional) - Maximum size of queue if using queues, 0 implies infinite hash: bool (optional) Whether or not to hash data to determine key. If False then this uses a random key @@ -2018,12 +1938,6 @@ def scatter( >>> c.scatter([1, 2, 3], workers=[('hostname', 8788)]) # doctest: +SKIP - Handle streaming sequences of data with iterators or queues - - >>> seq = c.scatter(iter([1, 2, 3])) # doctest: +SKIP - >>> next(seq) # doctest: +SKIP - , - Broadcast data to all workers >>> [future] = c.scatter([element], broadcast=True) # doctest: +SKIP @@ -2041,38 +1955,26 @@ def scatter( if timeout == no_default: timeout = self._timeout if isqueue(data) or isinstance(data, Iterator): - logger.debug("Starting thread for streaming data") - qout = pyQueue(maxsize=maxsize) - - t = threading.Thread( - target=self._threaded_scatter, - name="Threaded scatter()", - args=(data, qout), - kwargs={"workers": workers, "broadcast": broadcast}, + raise TypeError( + "Dask no longer supports mapping over Iterators or Queues." + "Consider using a normal for loop and Client.submit" ) - t.daemon = True - t.start() - if isqueue(data): - return qout - else: - return queue_to_iterator(qout) + if hasattr(thread_state, "execution_state"): # within worker task + local_worker = thread_state.execution_state["worker"] else: - if hasattr(thread_state, "execution_state"): # within worker task - local_worker = thread_state.execution_state["worker"] - else: - local_worker = None - return self.sync( - self._scatter, - data, - workers=workers, - broadcast=broadcast, - direct=direct, - local_worker=local_worker, - timeout=timeout, - asynchronous=asynchronous, - hash=hash, - ) + local_worker = None + return self.sync( + self._scatter, + data, + workers=workers, + broadcast=broadcast, + direct=direct, + local_worker=local_worker, + timeout=timeout, + asynchronous=asynchronous, + hash=hash, + ) @gen.coroutine def _cancel(self, futures, force=False): diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 5144c0f086..449d207a91 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -5,7 +5,6 @@ from collections import deque from concurrent.futures import CancelledError import gc -import itertools import logging import os import pickle @@ -52,7 +51,7 @@ futures_of, temp_default_client, ) -from distributed.compatibility import PY3, Iterator +from distributed.compatibility import PY3 from distributed.metrics import time from distributed.scheduler import Scheduler, KilledWorker @@ -1321,104 +1320,6 @@ def test_directed_scatter_sync(c, s, a, b, loop): assert len(has_what[a["address"]]) == 0 -def test_iterator_scatter(c): - aa = c.scatter([1, 2, 3]) - assert [1, 2, 3] == c.gather(aa) - - g = (i for i in range(10)) - futures = c.scatter(g) - assert isinstance(futures, Iterator) - - a = next(futures) - assert c.gather(a) == 0 - - futures = list(futures) - assert len(futures) == 9 - assert c.gather(futures) == [1, 2, 3, 4, 5, 6, 7, 8, 9] - - -def test_queue_scatter(c): - from distributed.compatibility import Queue - - q = Queue() - for d in range(10): - q.put(d) - - futures = c.scatter(q) - assert isinstance(futures, Queue) - a = futures.get() - assert c.gather(a) == 0 - - -def test_queue_scatter_gather_maxsize(c): - from distributed.compatibility import Queue - - q = Queue(maxsize=3) - out = c.scatter(q, maxsize=10) - assert out.maxsize == 10 - local = c.gather(q) - assert not local.maxsize - - q = Queue() - out = c.scatter(q) - assert not out.maxsize - local = c.gather(out, maxsize=10) - assert local.maxsize == 10 - - q = Queue(maxsize=3) - out = c.scatter(q) - assert not out.maxsize - - -def test_queue_gather(c): - from distributed.compatibility import Queue - - q = Queue() - - qin = list(range(10)) - for d in qin: - q.put(d) - - futures = c.scatter(q) - assert isinstance(futures, Queue) - - ff = c.gather(futures) - assert isinstance(ff, Queue) - - qout = [] - for f in range(10): - qout.append(ff.get()) - assert qout == qin - - -@pytest.mark.skip(reason="intermittent blocking failures") -def test_iterator_gather(c, c2): - i_in = list(range(10)) - - g = (d for d in i_in) - futures = c.scatter(g) - assert isinstance(futures, Iterator) - - ff = c.gather(futures) - assert isinstance(ff, Iterator) - - i_out = list(ff) - assert i_out == i_in - - i_in = ["a", "b", "c", StopIteration("f"), StopIteration, "d", "c"] - - g = (d for d in i_in) - futures = c.scatter(g) - - ff = c.gather(futures) - i_out = list(ff) - assert i_out[:3] == i_in[:3] - # This is because StopIteration('f') != StopIteration('f') - assert isinstance(i_out[3], StopIteration) - assert i_out[3].args == i_in[3].args - assert i_out[4:] == i_in[4:] - - @gen_cluster(client=True) def test_scatter_direct(c, s, a, b): future = yield c.scatter(123, direct=True) @@ -2373,109 +2274,6 @@ def test_traceback_clean(c, s, a, b): tb = tb.tb_next -@gen_cluster(client=True) -def test_map_queue(c, s, a, b): - from distributed.compatibility import Queue, isqueue - - q_1 = Queue(maxsize=2) - q_2 = c.map(inc, q_1) - assert isqueue(q_2) - assert not q_2.maxsize - q_3 = c.map(double, q_2, maxsize=3) - assert isqueue(q_3) - assert q_3.maxsize == 3 - q_4 = yield c._gather(q_3) - assert isqueue(q_4) - - q_1.put(1) - - f = q_4.get() - assert isinstance(f, Future) - result = yield f - assert result == (1 + 1) * 2 - - -@pytest.mark.skipif( - sys.version_info >= (3, 7), reason="replace StopIteration with return" -) -@gen_cluster(client=True) -def test_map_iterator_with_return(c, s, a, b): - def g(): - yield 1 - yield 2 - raise StopIteration(3) # py2.7 compat. - - f1 = c.map(lambda x: x, g()) - assert isinstance(f1, Iterator) - - start = time() # ensure that we compute eagerly - while not s.tasks: - yield gen.sleep(0.01) - assert time() < start + 5 - - g1 = g() - try: - while True: - f = next(f1) - n = yield f - assert n == next(g1) - except StopIteration as e: - with pytest.raises(StopIteration) as exc_info: - next(g1) - assert e.args == exc_info.value.args - - -@gen_cluster(client=True) -def test_map_iterator(c, s, a, b): - x = iter([1, 2, 3]) - y = iter([10, 20, 30]) - f1 = c.map(add, x, y) - assert isinstance(f1, Iterator) - - start = time() # ensure that we compute eagerly - while not s.tasks: - yield gen.sleep(0.01) - assert time() < start + 5 - - f2 = c.map(double, f1) - assert isinstance(f2, Iterator) - - future = next(f2) - result = yield future - assert result == (1 + 10) * 2 - futures = list(f2) - results = [] - for f in futures: - r = yield f - results.append(r) - assert results == [(2 + 20) * 2, (3 + 30) * 2] - - items = enumerate(range(10)) - futures = c.map(lambda x: x, items) - assert isinstance(futures, Iterator) - - result = yield next(futures) - assert result == (0, 0) - futures_l = list(futures) - results = [] - for f in futures_l: - r = yield f - results.append(r) - assert results == [(i, i) for i in range(1, 10)] - - -@gen_cluster(client=True) -def test_map_infinite_iterators(c, s, a, b): - futures = c.map(add, [1, 2], itertools.repeat(10)) - assert len(futures) == 2 - - -def test_map_iterator_sync(c): - items = enumerate(range(10)) - futures = c.map(lambda x: x, items) - next(futures).result() == (0, 0) - - @gen_cluster(client=True) def test_map_differnet_lengths(c, s, a, b): assert len(c.map(add, [1, 2], [1, 2, 3])) == 2 @@ -3557,7 +3355,7 @@ def test_get_stops_work_after_error(c): def test_as_completed_list(c): - seq = c.map(inc, iter(range(5))) + seq = c.map(inc, range(5)) seq2 = list(as_completed(seq)) assert set(c.gather(seq2)) == {1, 2, 3, 4, 5} diff --git a/distributed/tests/test_utils.py b/distributed/tests/test_utils.py index f4423d26e4..b82dce4e7d 100644 --- a/distributed/tests/test_utils.py +++ b/distributed/tests/test_utils.py @@ -14,7 +14,7 @@ from tornado.ioloop import IOLoop import dask -from distributed.compatibility import Queue, Empty, isqueue, PY2, Iterator +from distributed.compatibility import Queue, Empty, PY2 from distributed.metrics import time from distributed.utils import ( All, @@ -24,8 +24,6 @@ str_graph, truncate_exception, get_traceback, - queue_to_iterator, - iterator_to_queue, _maybe_complex, read_block, seek_delimiter, @@ -183,25 +181,6 @@ def c(x): assert type(tb).__name__ == "traceback" -def test_queue_to_iterator(): - q = Queue() - q.put(1) - q.put(2) - - seq = queue_to_iterator(q) - assert isinstance(seq, Iterator) - assert next(seq) == 1 - assert next(seq) == 2 - - -def test_iterator_to_queue(): - seq = iter([1, 2, 3]) - - q = iterator_to_queue(seq) - assert isqueue(q) - assert q.get() == 1 - - def test_str_graph(): dsk = {"x": 1} assert str_graph(dsk) == dsk diff --git a/distributed/utils.py b/distributed/utils.py index 765035e5c1..6debdedd24 100644 --- a/distributed/utils.py +++ b/distributed/utils.py @@ -46,7 +46,7 @@ except ImportError: PollIOLoop = None # dropped in tornado 6.0 -from .compatibility import Queue, PY3, PY2, get_thread_identity, unicode +from .compatibility import PY3, PY2, get_thread_identity, unicode from .metrics import time @@ -797,42 +797,6 @@ def truncate_exception(e, n=10000): return e -if sys.version_info >= (3,): - # (re-)raising StopIteration is deprecated in 3.6+ - exec( - """def queue_to_iterator(q): - while True: - result = q.get() - if isinstance(result, StopIteration): - return result.value - yield result - """ - ) -else: - # Returning non-None from generator is a syntax error in 2.x - def queue_to_iterator(q): - while True: - result = q.get() - if isinstance(result, StopIteration): - raise result - yield result - - -def _dump_to_queue(seq, q): - for item in seq: - q.put(item) - - -def iterator_to_queue(seq, maxsize=0): - q = Queue(maxsize=maxsize) - - t = threading.Thread(target=_dump_to_queue, args=(seq, q)) - t.daemon = True - t.start() - - return q - - def tokey(o): """ Convert an object to a string. diff --git a/distributed/utils_test.py b/distributed/utils_test.py index e40912fcb3..7b90745ac6 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -399,7 +399,7 @@ def map_varying(itemslists): def apply(func, *args, **kwargs): return func(*args, **kwargs) - return apply, map(varying, itemslists) + return apply, list(map(varying, itemslists)) @gen.coroutine diff --git a/docs/source/queues.rst b/docs/source/queues.rst index ea1bc76a4f..34a4cae538 100644 --- a/docs/source/queues.rst +++ b/docs/source/queues.rst @@ -1,194 +1,10 @@ Data Streams with Queues ======================== -The ``Client`` methods ``scatter``, ``map``, and ``gather`` can consume and -produce standard Python ``Queue`` objects. This is useful for processing -continuous streams of data. However, it does not constitute a full streaming -data processing pipeline like Storm. +This feature is no longer supported. +Instead people may want to look at the following options: -.. raw:: html - - - -Example -------- - -We connect to a local Client. - -.. code-block:: python - - >>> from distributed import Client - >>> client = Client('127.0.0.1:8786') - >>> client - - -We build a couple of toy data processing functions: - -.. code-block:: python - - from time import sleep - from random import random - - def inc(x): - from random import random - sleep(random() * 2) - return x + 1 - - def double(x): - from random import random - sleep(random()) - return 2 * x - -And we set up an input Queue and map our functions across it. - -.. code-block:: python - - >>> from queue import Queue - >>> input_q = Queue() - >>> remote_q = client.scatter(input_q) - >>> inc_q = client.map(inc, remote_q) - >>> double_q = client.map(double, inc_q) - -We will fill the ``input_q`` with local data from some stream, and then -``remote_q``, ``inc_q`` and ``double_q`` will fill with ``Future`` objects as -data gets moved around. - -We gather the futures from the ``double_q`` back to a queue holding local -data in the local process. - -.. code-block:: python - - >>> result_q = client.gather(double_q) - -Insert Data Manually -~~~~~~~~~~~~~~~~~~~~ - -Because we haven't placed any data into any of the queues everything is empty, -including the final output, ``result_q``. - -.. code-block:: python - - >>> result_q.qsize() - 0 - -But when we insert an entry into the ``input_q``, it starts to make its way -through the pipeline and ends up in the ``result_q``. - -.. code-block:: python - - >>> input_q.put(10) - >>> result_q.get() - 22 - -Insert data in a separate thread -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -We simulate a slightly more realistic situation by dumping data into the -``input_q`` in a separate thread. This simulates what you might get if you -were to read from an active data source. - -.. code-block:: python - - def load_data(q): - i = 0 - while True: - q.put(i) - sleep(random()) - i += 1 - - >>> from threading import Thread - >>> load_thread = Thread(target=load_data, args=(input_q,)) - >>> load_thread.start() - - >>> result_q.qsize() - 4 - >>> result_q.qsize() - 9 - -We consume data from the ``result_q`` and print results to the screen. - -.. code-block:: python - - >>> while True: - ... item = result_q.get() - ... print(item) - 2 - 4 - 6 - 8 - 10 - 12 - ... - -Limitations ------------ - -* This doesn't do any sort of auto-batching of computations, so ideally you - batch your data to take significantly longer than 1ms to run. -* This isn't a proper streaming system. There is no support outside of what - you see here. In particular there are no policies for dropping data, joining - over time windows, etc.. - -Extensions ----------- - -We can extend this small example to more complex systems that have buffers, -split queues, merge queues, etc. all by manipulating normal Python Queues. - -Here are a couple of useful function to multiplex and merge queues: - -.. code-block:: python - - from queue import Queue - from threading import Thread - - def multiplex(n, q, **kwargs): - """ Convert one queue into several equivalent Queues - - >>> q1, q2, q3 = multiplex(3, in_q) - """ - out_queues = [Queue(**kwargs) for i in range(n)] - def f(): - while True: - x = q.get() - for out_q in out_queues: - out_q.put(x) - t = Thread(target=f) - t.daemon = True - t.start() - return out_queues - - def push(in_q, out_q): - while True: - x = in_q.get() - out_q.put(x) - - def merge(*in_qs, **kwargs): - """ Merge multiple queues together - - >>> out_q = merge(q1, q2, q3) - """ - out_q = Queue(**kwargs) - threads = [Thread(target=push, args=(q, out_q)) for q in in_qs] - for t in threads: - t.daemon = True - t.start() - return out_q - -With useful functions like these we can build out more sophisticated data -processing pipelines that split off and join back together. By creating queues -with ``maxsize=`` we can control buffering and apply back pressure. - -.. raw:: html - - +1. Use normal for loops with Client.submit/gather and as_completed +2. Use :doc:`asynchronous async/await ` code and a few coroutines +3. Try out the `Streamz `_ project, + which has Dask support From bb80c5e997d020c5b472819fd0751c8234d2b8cc Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 13 May 2019 18:19:59 -0500 Subject: [PATCH 06/43] Use 'temporary-directory' from dask.config for Worker's directory (#2654) --- distributed/tests/test_worker.py | 9 +++++++++ distributed/worker.py | 8 +++++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/distributed/tests/test_worker.py b/distributed/tests/test_worker.py index d8ca4d3148..4541e183e4 100644 --- a/distributed/tests/test_worker.py +++ b/distributed/tests/test_worker.py @@ -1383,6 +1383,15 @@ def __init__(self, x, y): yield w.close() +@gen_cluster(ncores=[]) +def test_local_dir(s): + with tmpfile() as fn: + with dask.config.set(temporary_directory=fn): + w = yield Worker(s.address) + assert w.local_dir.startswith(fn) + assert "dask-worker-space" in w.local_dir + + @pytest.mark.skipif( not sys.platform.startswith("linux"), reason="Need 127.0.0.2 to mean localhost" ) diff --git a/distributed/worker.py b/distributed/worker.py index 8b8c139f35..3b5e1cb233 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -282,7 +282,7 @@ def __init__( scheduler_file=None, ncores=None, loop=None, - local_dir="dask-worker-space", + local_dir=None, services=None, service_ports=None, service_kwargs=None, @@ -448,6 +448,12 @@ def __init__( if silence_logs: silence_logging(level=silence_logs) + if local_dir is None: + local_dir = dask.config.get("temporary-directory") or os.getcwd() + if not os.path.exists(local_dir): + os.mkdir(local_dir) + local_dir = os.path.join(local_dir, "dask-worker-space") + with warn_on_duration( "1s", "Creating scratch directories is taking a surprisingly long time. " From fd31ecca8017bae845a73d468de0376c02363fab Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Tue, 14 May 2019 12:49:49 -0500 Subject: [PATCH 07/43] Cleanup localcluster (#2693) * Remove address handling (handled in scheduler) * Move ip= keyword to host= --- distributed/deploy/local.py | 57 +++++++++++--------------- distributed/deploy/tests/test_local.py | 14 ++++--- 2 files changed, 33 insertions(+), 38 deletions(-) diff --git a/distributed/deploy/local.py b/distributed/deploy/local.py index fb8793d084..832e8f3e05 100644 --- a/distributed/deploy/local.py +++ b/distributed/deploy/local.py @@ -15,7 +15,6 @@ from ..compatibility import get_thread_identity from ..core import CommClosedError from ..utils import ( - get_ip_interface, sync, ignoring, All, @@ -51,8 +50,10 @@ class LocalCluster(Cluster): silence_logs: logging level Level of logs to print out to stdout. ``logging.WARN`` by default. Use a falsey value like False or None for no change. + host: string + Host address on which the scheduler will listen, defaults to only localhost ip: string - IP address on which the scheduler will listen, defaults to only localhost + Deprecated. See ``host`` above. dashboard_address: str Address on which to listen for the Bokeh diagnostics server like 'localhost:8787' or '0.0.0.0:8787'. Defaults to ':8787'. @@ -88,13 +89,9 @@ class LocalCluster(Cluster): >>> c = Client(cluster) # connect to local cluster # doctest: +SKIP - Add a new worker to the cluster + Scale the cluster to three workers - >>> w = cluster.start_worker(ncores=2) # doctest: +SKIP - - Shut down the extra worker - - >>> cluster.stop_worker(w) # doctest: +SKIP + >>> cluster.scale(3) # doctest: +SKIP Pass extra keyword arguments to Bokeh @@ -109,6 +106,7 @@ def __init__( loop=None, start=None, ip=None, + host=None, scheduler_port=0, silence_logs=logging.WARN, dashboard_address=":8787", @@ -125,6 +123,10 @@ def __init__( worker_class=None, **worker_kwargs ): + if ip is not None: + warnings.warn("The ip keyword has been moved to host") + host = ip + if start is not None: msg = ( "The start= parameter is deprecated. " @@ -145,8 +147,8 @@ def __init__( self.processes = processes if protocol is None: - if ip and "://" in ip: - protocol = ip.split("://")[0] + if host and "://" in host: + protocol = host.split("://")[0] elif security: protocol = "tls://" elif not self.processes and not scheduler_port: @@ -155,12 +157,12 @@ def __init__( protocol = "tcp://" if not protocol.endswith("://"): protocol = protocol + "://" - self.protocol = protocol + + if host is None and not protocol.startswith("inproc") and not interface: + host = "127.0.0.1" self.silence_logs = silence_logs self._asynchronous = asynchronous - self.security = security - self.interface = interface services = services or {} worker_services = worker_services or {} if silence_logs: @@ -184,6 +186,8 @@ def __init__( "ncores": threads_per_worker, "services": worker_services, "dashboard_address": worker_dashboard_address, + "interface": interface, + "protocol": protocol, } ) @@ -192,14 +196,16 @@ def __init__( self.scheduler = Scheduler( loop=self.loop, + host=host, services=services, service_kwargs=service_kwargs, security=security, + port=scheduler_port, interface=interface, + protocol=protocol, dashboard_address=dashboard_address, blocked_handlers=blocked_handlers, ) - self.scheduler_port = scheduler_port self.workers = [] self.worker_kwargs = worker_kwargs @@ -210,7 +216,7 @@ def __init__( worker_class = Worker if not processes else Nanny self.worker_class = worker_class - self.start(ip=ip, n_workers=n_workers) + self.start(n_workers=n_workers) clusters_to_close.add(self) @@ -251,32 +257,17 @@ def start(self, **kwargs): self.sync(self._start, **kwargs) @gen.coroutine - def _start(self, ip=None, n_workers=0): + def _start(self, n_workers=0): """ Start all cluster services. """ if self.status == "running": return - if self.protocol == "inproc://": - address = self.protocol - else: - if ip is None: - if self.interface: - ip = get_ip_interface(self.interface) - else: - ip = "127.0.0.1" - - if "://" in ip: - address = ip - else: - address = self.protocol + ip - if self.scheduler_port: - address += ":" + str(self.scheduler_port) - - self.scheduler.start(address) + self.scheduler.start() yield [self._start_worker(**self.worker_kwargs) for i in range(n_workers)] + yield self.scheduler self.status = "running" diff --git a/distributed/deploy/tests/test_local.py b/distributed/deploy/tests/test_local.py index 6e1e71e83b..ed9e3bb2db 100644 --- a/distributed/deploy/tests/test_local.py +++ b/distributed/deploy/tests/test_local.py @@ -128,7 +128,7 @@ def test_move_unserializable_data(): assert y.result() is lock -def test_transports(): +def test_transports_inproc(): """ Test the transport chosen by LocalCluster depending on arguments. """ @@ -140,6 +140,8 @@ def test_transports(): with Client(c.scheduler.address) as e: assert e.submit(inc, 4).result() == 5 + +def test_transports_tcp(): # Have nannies => need TCP with LocalCluster( 1, processes=True, silence_logs=False, dashboard_address=None @@ -149,6 +151,8 @@ def test_transports(): with Client(c.scheduler.address) as e: assert e.submit(inc, 4).result() == 5 + +def test_transports_tcp_port(): # Scheduler port specified => need TCP with LocalCluster( 1, @@ -417,7 +421,7 @@ def test_remote_access(loop): scheduler_port=0, silence_logs=False, dashboard_address=None, - ip="", + host="", loop=loop, ) as c: sync(loop, assert_can_connect_from_everywhere_4_6, c.scheduler.port) @@ -620,7 +624,7 @@ def test_local_tls(loop): silence_logs=False, security=security, dashboard_address=False, - ip="tls://0.0.0.0", + host="tls://0.0.0.0", loop=loop, ) as c: sync( @@ -690,7 +694,7 @@ def test_local_tls_restart(loop): silence_logs=False, security=security, dashboard_address=False, - ip="tls://0.0.0.0", + host="tls://0.0.0.0", loop=loop, ) as c: with Client(c.scheduler.address, loop=loop, security=security) as client: @@ -750,7 +754,7 @@ def test_protocol_tcp(loop): ) def test_protocol_ip(loop): with LocalCluster( - ip="tcp://127.0.0.2", loop=loop, n_workers=0, processes=False + host="tcp://127.0.0.2", loop=loop, n_workers=0, processes=False ) as cluster: assert cluster.scheduler.address.startswith("tcp://127.0.0.2") From 8e449d392e91eff0a3454ee98ef362de8f78cc4f Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Wed, 15 May 2019 09:35:43 -0500 Subject: [PATCH 08/43] Support computation on delayed(None) (#2697) Previously this conflicted with our sentinel value Fixes #2696 --- distributed/tests/test_collections.py | 10 ++++++++++ distributed/worker.py | 6 +++--- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/distributed/tests/test_collections.py b/distributed/tests/test_collections.py index 985b6f78fe..dea4296769 100644 --- a/distributed/tests/test_collections.py +++ b/distributed/tests/test_collections.py @@ -6,6 +6,7 @@ pytest.importorskip("numpy") pytest.importorskip("pandas") +import dask import dask.dataframe as dd import dask.bag as db from distributed.client import wait @@ -185,3 +186,12 @@ def test_sparse_arrays(c, s, a, b): future = c.compute(s.sum(axis=0)[:10]) yield future + + +@gen_cluster(client=True, ncores=[("127.0.0.1", 1)]) +def test_delayed_none(c, s, w): + x = dask.delayed(None) + y = dask.delayed(123) + [xx, yy] = c.compute([x, y]) + assert (yield xx) is None + assert (yield yy) == 123 diff --git a/distributed/worker.py b/distributed/worker.py index 3b5e1cb233..f418939309 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -1216,7 +1216,7 @@ def add_task( function=None, args=None, kwargs=None, - task=None, + task=no_value, who_has=None, nbytes=None, priority=None, @@ -3007,7 +3007,7 @@ def get_data_from_worker( job_counter = [0] -def _deserialize(function=None, args=None, kwargs=None, task=None): +def _deserialize(function=None, args=None, kwargs=None, task=no_value): """ Deserialize task inputs and regularize to func, args, kwargs """ if function is not None: function = pickle.loads(function) @@ -3016,7 +3016,7 @@ def _deserialize(function=None, args=None, kwargs=None, task=None): if kwargs: kwargs = pickle.loads(kwargs) - if task is not None: + if task is not no_value: assert not function and not args and not kwargs function = execute_task args = (task,) From 73362eaf4bb1f941284658ae87f51dbaae753a4c Mon Sep 17 00:00:00 2001 From: Daniel Farrell Date: Wed, 15 May 2019 10:04:30 -0700 Subject: [PATCH 09/43] Add method to wait for n workers before continuing (#2688) --- distributed/client.py | 11 +++++++++++ distributed/tests/test_client.py | 13 +++++++++++++ 2 files changed, 24 insertions(+) diff --git a/distributed/client.py b/distributed/client.py index 7b6e14aa4e..8dd85d7795 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -1051,6 +1051,17 @@ def _update_scheduler_info(self): except EnvironmentError: logger.debug("Not able to query scheduler for identity") + @gen.coroutine + def _wait_for_workers(self, n_workers=0): + info = yield self.scheduler.identity() + while n_workers and len(info["workers"]) < n_workers: + yield gen.sleep(0.1) + info = yield self.scheduler.identity() + + def wait_for_workers(self, n_workers=0): + """Blocking call to wait for n workers before continuing""" + return self.sync(self._wait_for_workers, n_workers) + def _heartbeat(self): if self.scheduler_comm: self.scheduler_comm.send({"op": "heartbeat-client"}) diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 449d207a91..dfe677ddd1 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -5510,5 +5510,18 @@ def test_instances(c, s, a, b): assert set(Worker._instances) == {a, b} +@gen_cluster(client=True) +def test_wait_for_workers(c, s, a, b): + future = c.wait_for_workers(n_workers=3) + yield gen.sleep(0.22) # 2 chances + assert not future.done() + + w = yield Worker(s.address) + start = time() + yield future + assert time() < start + 1 + yield w.close() + + if sys.version_info >= (3, 5): from distributed.tests.py3_test_client import * # noqa F401 From 0ce8f2bcb84d306e9a095d75497857dce30145b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mathieu=20Dugr=C3=A9?= Date: Wed, 15 May 2019 15:56:44 -0400 Subject: [PATCH 10/43] Modify styling of histograms for many-worker dashboard plots (#2695) Fixes #2691 --- distributed/bokeh/scheduler.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/distributed/bokeh/scheduler.py b/distributed/bokeh/scheduler.py index 2dd60f0690..c078b612dd 100644 --- a/distributed/bokeh/scheduler.py +++ b/distributed/bokeh/scheduler.py @@ -219,9 +219,10 @@ def __init__(self, scheduler, **kwargs): ) self.root = figure( - title="Tasks Processing", + title="Tasks Processing (Histogram)", id="bk-nprocessing-histogram-plot", name="processing_hist", + y_axis_label="frequency", **kwargs ) @@ -237,7 +238,8 @@ def __init__(self, scheduler, **kwargs): right="right", bottom=0, top="top", - color="blue", + color="deepskyblue", + fill_alpha=0.5, ) @without_property_validation @@ -259,11 +261,13 @@ def __init__(self, scheduler, **kwargs): ) self.root = figure( - title="Bytes Stored", + title="Bytes Stored (Histogram)", name="nbytes_hist", id="bk-nbytes-histogram-plot", + y_axis_label="frequency", **kwargs ) + self.root.xaxis[0].formatter = NumeralTickFormatter(format="0.0 b") self.root.xaxis.major_label_orientation = -math.pi / 12 @@ -279,7 +283,8 @@ def __init__(self, scheduler, **kwargs): right="right", bottom=0, top="top", - color="blue", + color="deepskyblue", + fill_alpha=0.5, ) @without_property_validation @@ -289,7 +294,7 @@ def update(self): d = {"left": x[:-1], "right": x[1:], "top": counts} self.source.data.update(d) - self.root.title.text = "Bytes stored: " + format_bytes(nbytes.sum()) + self.root.title.text = "Bytes stored (Histogram): " + format_bytes(nbytes.sum()) class CurrentLoad(DashboardComponent): From d4f478672137481ad43abc6a44383a595e8485b5 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Thu, 16 May 2019 08:23:42 -0500 Subject: [PATCH 11/43] Handle heartbeat when worker has just left (#2702) --- distributed/scheduler.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 446b479c76..1d6a41a9ac 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1353,6 +1353,9 @@ def heartbeat_worker( address = self.coerce_address(address, resolve_address) address = normalize_address(address) host = get_address_host(address) + if address not in self.workers: + logger.info("Received heartbeat from removed worker: %s", address) + return local_now = time() now = now or time() From 4feb90d7ddd4860648c161c1497bb55da0fc1b2a Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Thu, 16 May 2019 08:23:54 -0500 Subject: [PATCH 12/43] Except errors in Nanny's memory monitor if process no longer exists (#2701) --- distributed/nanny.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/distributed/nanny.py b/distributed/nanny.py index 7be630cb4e..842ec765d7 100644 --- a/distributed/nanny.py +++ b/distributed/nanny.py @@ -326,9 +326,9 @@ def memory_monitor(self): return try: proc = psutil.Process(process.pid) - except psutil.NoSuchProcess: + memory = proc.memory_info().rss + except (ProcessLookupError, psutil.NoSuchProcess, psutil.AccessDenied): return - memory = proc.memory_info().rss frac = memory / self.memory_limit if self.memory_terminate_fraction and frac > self.memory_terminate_fraction: logger.warning( From d9a0897cd3abc6f0c921f50b5e12dbb10fc5aac5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mathieu=20Dugr=C3=A9?= Date: Thu, 16 May 2019 16:25:15 -0400 Subject: [PATCH 13/43] Disable pan tool for the Progress, Byte Stored and Tasks Processing plot (#2703) --- distributed/bokeh/scheduler.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/distributed/bokeh/scheduler.py b/distributed/bokeh/scheduler.py index c078b612dd..471f93dd4b 100644 --- a/distributed/bokeh/scheduler.py +++ b/distributed/bokeh/scheduler.py @@ -223,6 +223,7 @@ def __init__(self, scheduler, **kwargs): id="bk-nprocessing-histogram-plot", name="processing_hist", y_axis_label="frequency", + tools="", **kwargs ) @@ -265,6 +266,7 @@ def __init__(self, scheduler, **kwargs): name="nbytes_hist", id="bk-nbytes-histogram-plot", y_axis_label="frequency", + tools="", **kwargs ) @@ -943,6 +945,7 @@ def __init__(self, scheduler, **kwargs): x_range=x_range, y_range=y_range, toolbar_location=None, + tools="", **kwargs ) self.root.line( # just to define early ranges From a42721656418ff4848c44e230b8f033d2db58a63 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Thu, 16 May 2019 16:36:07 -0500 Subject: [PATCH 14/43] Cleanly stop periodic callbacks in Client (#2705) Previously we did this only in the asynchronous code, which left a gap during which a heartbeat could sneak out. Now we call it explicitly at the beginning of the synchronous close command. --- distributed/client.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/distributed/client.py b/distributed/client.py index 8dd85d7795..4aab8102d0 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -1204,10 +1204,11 @@ def _close(self, fast=False): """ Send close signal and wait until scheduler completes """ self.status = "closing" + for pc in self._periodic_callbacks.values(): + pc.stop() + with log_errors(): _del_global_client(self) - for pc in self._periodic_callbacks.values(): - pc.stop() self._scheduler_identity = {} with ignoring(AttributeError): # clear the dask.config set keys @@ -1289,6 +1290,9 @@ def close(self, timeout=no_default): return self.status = "closing" + for pc in self._periodic_callbacks.values(): + pc.stop() + if self.asynchronous: future = self._close() if timeout: From 7ebe65980e7fef90fd25cc0d35e2fcfc0c266881 Mon Sep 17 00:00:00 2001 From: Benjamin Zaitlen Date: Thu, 16 May 2019 17:36:36 -0400 Subject: [PATCH 15/43] Change the main workers bokeh page to /status (#2689) This matches the behavior on the scheduler --- distributed/bokeh/scheduler.py | 4 ++-- distributed/bokeh/worker.py | 16 ++++++++-------- distributed/bokeh/worker_html.py | 11 ++++++++++- distributed/cli/tests/test_dask_worker.py | 4 +++- docs/source/diagnosing-performance.rst | 2 +- 5 files changed, 24 insertions(+), 13 deletions(-) diff --git a/distributed/bokeh/scheduler.py b/distributed/bokeh/scheduler.py index 471f93dd4b..cce94b356a 100644 --- a/distributed/bokeh/scheduler.py +++ b/distributed/bokeh/scheduler.py @@ -152,7 +152,7 @@ def __init__(self, scheduler, **kwargs): # fig.xaxis[0].formatter = NumeralTickFormatter(format='0.0s') fig.x_range.start = 0 - tap = TapTool(callback=OpenURL(url="http://@bokeh_address/main")) + tap = TapTool(callback=OpenURL(url="http://@bokeh_address/")) hover = HoverTool() hover.tooltips = "@worker : @occupancy s." @@ -368,7 +368,7 @@ def __init__(self, scheduler, width=600, **kwargs): fig.yaxis.visible = False fig.ygrid.visible = False - tap = TapTool(callback=OpenURL(url="http://@bokeh_address/main")) + tap = TapTool(callback=OpenURL(url="http://@bokeh_address/")) fig.add_tools(tap) fig.toolbar.logo = None diff --git a/distributed/bokeh/worker.py b/distributed/bokeh/worker.py index c7ced4d90f..ed7b68b76b 100644 --- a/distributed/bokeh/worker.py +++ b/distributed/bokeh/worker.py @@ -51,7 +51,7 @@ BOKEH_THEME = Theme(os.path.join(os.path.dirname(__file__), "theme.yaml")) -template_variables = {"pages": ["main", "system", "profile", "crossfilter"]} +template_variables = {"pages": ["status", "system", "profile", "crossfilter"]} class StateTable(DashboardComponent): @@ -410,9 +410,9 @@ def process_msg(self, msg): def func(k): return msg["keys"].get(k, 0) - main_key = max(msg["keys"], key=func) - typ = self.worker.types.get(main_key, object).__name__ - keyname = key_split(main_key) + status_key = max(msg["keys"], key=func) + typ = self.worker.types.get(status_key, object).__name__ + keyname = key_split(status_key) d = { "nbytes": msg["total"], "duration": msg["duration"], @@ -659,7 +659,7 @@ def update(self): from bokeh.application import Application -def main_doc(worker, extra, doc): +def status_doc(worker, extra, doc): with log_errors(): statetable = StateTable(worker) executing_ts = ExecutingTimeSeries(worker, sizing_mode="scale_width") @@ -685,7 +685,7 @@ def main_doc(worker, extra, doc): ) ) doc.template = env.get_template("simple.html") - doc.template_variables["active_page"] = "main" + doc.template_variables["active_page"] = "status" doc.template_variables.update(extra) doc.theme = BOKEH_THEME @@ -773,7 +773,7 @@ def __init__(self, worker, io_loop=None, prefix="", **kwargs): extra.update(template_variables) - main = Application(FunctionHandler(partial(main_doc, worker, extra))) + status = Application(FunctionHandler(partial(status_doc, worker, extra))) crossfilter = Application( FunctionHandler(partial(crossfilter_doc, worker, extra)) ) @@ -787,7 +787,7 @@ def __init__(self, worker, io_loop=None, prefix="", **kwargs): ) self.apps = { - "/main": main, + "/status": status, "/counters": counters, "/crossfilter": crossfilter, "/system": systemmonitor, diff --git a/distributed/bokeh/worker_html.py b/distributed/bokeh/worker_html.py index 3ddf9490c4..c818c8fb1e 100644 --- a/distributed/bokeh/worker_html.py +++ b/distributed/bokeh/worker_html.py @@ -67,7 +67,16 @@ def get(self): self.set_header("Content-Type", "text/plain") -routes = [(r"metrics", PrometheusHandler), (r"health", HealthHandler)] +class OldRoute(RequestHandler): + def get(self): + self.redirect("/status") + + +routes = [ + (r"metrics", PrometheusHandler), + (r"health", HealthHandler), + (r"main", OldRoute), +] def get_handlers(server): diff --git a/distributed/cli/tests/test_dask_worker.py b/distributed/cli/tests/test_dask_worker.py index c26c99f235..5ed668e758 100644 --- a/distributed/cli/tests/test_dask_worker.py +++ b/distributed/cli/tests/test_dask_worker.py @@ -260,8 +260,10 @@ def test_bokeh_non_standard_ports(loop): start = time() while True: try: - response = requests.get("http://127.0.0.1:4833/main") + response = requests.get("http://127.0.0.1:4833/status") assert response.ok + redirect_resp = requests.get("http://127.0.0.1:4833/main") + redirect_resp.ok break except Exception: sleep(0.5) diff --git a/docs/source/diagnosing-performance.rst b/docs/source/diagnosing-performance.rst index 28d7d9aba4..773a5d2316 100644 --- a/docs/source/diagnosing-performance.rst +++ b/docs/source/diagnosing-performance.rst @@ -105,7 +105,7 @@ attributes including 4. Keys moved 5. Peer -These are made available to users through the ``/main`` page of the Worker's +These are made available to users through the ``/status`` page of the Worker's diagnostic dashboard. You can capture their state explicitly by running a command on the workers: From fc48c435f8f366c335c92dd6fc58af38065edcec Mon Sep 17 00:00:00 2001 From: Sam Grayson Date: Fri, 17 May 2019 17:53:12 -0500 Subject: [PATCH 16/43] Support uploading files with multiple modules (#2587) --- distributed/tests/test_client.py | 96 ++++++++++++++++++++++++-------- distributed/utils.py | 15 ++--- distributed/utils_test.py | 15 +++++ 3 files changed, 94 insertions(+), 32 deletions(-) diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index dfe677ddd1..4cd196fa2f 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -9,6 +9,7 @@ import os import pickle import random +import subprocess import sys import threading from threading import Semaphore @@ -79,6 +80,7 @@ wait_for, async_wait_for, pristine_loop, + save_sys_modules, ) from distributed.utils_test import ( # noqa: F401 client as c, @@ -1484,7 +1486,7 @@ def g(): return myfile.f() - try: + with save_sys_modules(): for value in [123, 456]: with tmp_text("myfile.py", "def f():\n return {}".format(value)) as fn: yield c.upload_file(fn) @@ -1492,10 +1494,6 @@ def g(): x = c.submit(g, pure=False) result = yield x assert result == value - finally: - # Ensure that this test won't impact the others - if "myfile" in sys.modules: - del sys.modules["myfile"] @gen_cluster(client=True) @@ -1511,28 +1509,80 @@ def g(): return myfile.f() - try: + with save_sys_modules(): + try: + for value in [123, 456]: + with tmp_text( + "myfile.py", "def f():\n return {}".format(value) + ) as fn_my_file: + with zipfile.ZipFile("myfile.zip", "w") as z: + z.write(fn_my_file, arcname=os.path.basename(fn_my_file)) + yield c.upload_file("myfile.zip") + + x = c.submit(g, pure=False) + result = yield x + assert result == value + finally: + if os.path.exists("myfile.zip"): + os.remove("myfile.zip") + + +@gen_cluster(client=True) +def test_upload_file_egg(c, s, a, b): + def g(): + import package_1, package_2 + + return package_1.a, package_2.b + + # c.upload_file tells each worker to + # - put this file in their local_dir + # - modify their sys.path to include it + # we don't care about the local_dir + # but we do care about restoring the path + + with save_sys_modules(): for value in [123, 456]: - with tmp_text( - "myfile.py", "def f():\n return {}".format(value) - ) as fn_my_file: - with zipfile.ZipFile("myfile.zip", "w") as z: - z.write(fn_my_file, arcname=os.path.basename(fn_my_file)) - yield c.upload_file("myfile.zip") + with tmpfile() as dirname: + os.mkdir(dirname) + + with open(os.path.join(dirname, "setup.py"), "w") as f: + f.write("from setuptools import setup, find_packages\n") + f.write( + 'setup(name="my_package", packages=find_packages(), version="{}")\n'.format( + value + ) + ) + + # test a package with an underscore in the name + package_1 = os.path.join(dirname, "package_1") + os.mkdir(package_1) + with open(os.path.join(package_1, "__init__.py"), "w") as f: + f.write("a = {}\n".format(value)) + + # test multiple top-level packages + package_2 = os.path.join(dirname, "package_2") + os.mkdir(package_2) + with open(os.path.join(package_2, "__init__.py"), "w") as f: + f.write("b = {}\n".format(value)) + + # compile these into an egg + subprocess.check_call( + [sys.executable, "setup.py", "bdist_egg"], cwd=dirname + ) + + egg_root = os.path.join(dirname, "dist") + # first file ending with '.egg' + egg_name = [ + fname for fname in os.listdir(egg_root) if fname.endswith(".egg") + ][0] + egg_path = os.path.join(egg_root, egg_name) + + yield c.upload_file(egg_path) + os.remove(egg_path) x = c.submit(g, pure=False) result = yield x - assert result == value - finally: - # Ensure that this test won't impact the others - if os.path.exists("myfile.zip"): - os.remove("myfile.zip") - if "myfile" in sys.modules: - del sys.modules["myfile"] - for path in sys.path: - if os.path.basename(path) == "myfile.zip": - sys.path.remove(path) - break + assert result == (value, value) @gen_cluster(client=True) diff --git a/distributed/utils.py b/distributed/utils.py index 6debdedd24..466a96fdfd 100644 --- a/distributed/utils.py +++ b/distributed/utils.py @@ -23,7 +23,7 @@ import threading import warnings import weakref - +import pkgutil import six import tblib.pickling_support @@ -1066,14 +1066,11 @@ def import_file(path): if ext in (".egg", ".zip", ".pyz"): if path not in sys.path: sys.path.insert(0, path) - if ext == ".egg": - import pkg_resources - - pkgs = pkg_resources.find_distributions(path) - for pkg in pkgs: - names_to_import.append(pkg.project_name) - elif ext in (".zip", ".pyz"): - names_to_import.append(name) + if sys.version_info >= (3, 6): + names = (mod_info.name for mod_info in pkgutil.iter_modules([path])) + else: + names = (mod_info[1] for mod_info in pkgutil.iter_modules([path])) + names_to_import.extend(names) loaded = [] if not names_to_import: diff --git a/distributed/utils_test.py b/distributed/utils_test.py index 7b90745ac6..c44f417747 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -1417,6 +1417,21 @@ def gen_tls_cluster(**kwargs): ) +@contextmanager +def save_sys_modules(): + old_modules = sys.modules + old_path = sys.path + try: + yield + finally: + for i, elem in enumerate(sys.path): + if elem not in old_path: + del sys.path[i] + for elem in sys.modules.keys(): + if elem not in old_modules: + del sys.modules[elem] + + @contextmanager def check_thread_leak(): active_threads_start = set(threading._active) From 138842c9769c273b6edc6c086c80009102596198 Mon Sep 17 00:00:00 2001 From: Magnus Nord Date: Mon, 20 May 2019 17:54:37 +0200 Subject: [PATCH 17/43] Fix two typos in Pub class docstring (#2714) --- distributed/pubsub.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/distributed/pubsub.py b/distributed/pubsub.py index 5e08649292..f9cf1f6f7c 100644 --- a/distributed/pubsub.py +++ b/distributed/pubsub.py @@ -234,8 +234,8 @@ class Pub(object): disappear without notice. When using a Pub or Sub from a Client all communications will be routed - through the scheduler. This can cause some performance degredation. Pubs - an Subs only operate at top-speed when they are both on workers. + through the scheduler. This can cause some performance degradation. Pubs + and Subs only operate at top-speed when they are both on workers. Parameters ---------- From 1a96f70d9c2d23d85a8550f5bfdf60c26bb4ed4f Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 20 May 2019 12:12:52 -0500 Subject: [PATCH 18/43] Remove special casing of Scikit-Learn BaseEstimator serialization (#2713) Fixes https://github.com/dask/dask/issues/4769 --- distributed/protocol/__init__.py | 8 -------- distributed/protocol/tests/test_sklearn.py | 4 ++++ 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/distributed/protocol/__init__.py b/distributed/protocol/__init__.py index cf1a3df899..04691ce605 100644 --- a/distributed/protocol/__init__.py +++ b/distributed/protocol/__init__.py @@ -60,14 +60,6 @@ def _register_arrow(): from . import arrow -@dask_serialize.register_lazy("sklearn") -@dask_deserialize.register_lazy("sklearn") -def _register_sklearn(): - import sklearn.base - - register_generic(sklearn.base.BaseEstimator) - - @dask_serialize.register_lazy("torch") @dask_deserialize.register_lazy("torch") @dask_serialize.register_lazy("torchvision") diff --git a/distributed/protocol/tests/test_sklearn.py b/distributed/protocol/tests/test_sklearn.py index 051a0440f3..2a3835168e 100644 --- a/distributed/protocol/tests/test_sklearn.py +++ b/distributed/protocol/tests/test_sklearn.py @@ -7,6 +7,10 @@ from distributed.protocol import serialize, deserialize +@pytest.mark.xfail( + reason="We no longer special-case the BaseEstimator " + "super class. It's hard to guarantee support for all subclasseses" +) def test_basic(): est = sklearn.linear_model.LinearRegression() est.fit([[0, 0], [1, 1], [2, 2]], [0, 1, 2]) From f47ed2e610590c644da052af64a34bffa1552a92 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Tue, 21 May 2019 17:45:13 -0500 Subject: [PATCH 19/43] Refer to LocalCluster in Client docstring (#2719) --- distributed/client.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/distributed/client.py b/distributed/client.py index 4aab8102d0..6b5bc2811b 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -504,12 +504,19 @@ class AllExit(Exception): class Client(Node): - """ Connect to and drive computation on a distributed Dask cluster - - The Client connects users to a dask.distributed compute cluster. It - provides an asynchronous user interface around functions and futures. This - class resembles executors in ``concurrent.futures`` but also allows - ``Future`` objects within ``submit/map`` calls. + """ Connect to and submit computation to a Dask cluster + + The Client connects users to a Dask cluster. It provides an asynchronous + user interface around functions and futures. This class resembles + executors in ``concurrent.futures`` but also allows ``Future`` objects + within ``submit/map`` calls. When a Client is instantiated it takes over + all ``dask.compute`` and ``dask.persist`` calls by default. + + It is also common to create a Client without specifying the scheduler + address , like ``Client()``. In this case the Client creates a + ``LocalCluster`` in the background and connects to that. Any extra + keywords are passed from Client to LocalCluster in this case. See the + LocalCluster documentation for more information. Parameters ---------- From 62f604e7e567a1cc7806226adc5d7f288dc2fbad Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Wed, 22 May 2019 15:05:39 -0500 Subject: [PATCH 20/43] Add docstring to Scheduler.check_idle_saturated (#2721) --- distributed/scheduler.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 1d6a41a9ac..801c0c849d 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -4357,6 +4357,19 @@ def reschedule(self, key=None, worker=None): ############################## def check_idle_saturated(self, ws, occ=None): + """ Update the status of the idle and saturated state + + The scheduler keeps track of workers that are .. + + - Saturated: have enough work to stay busy + - Idle: do not have enough work to stay busy + + They are considered saturated if they both have enough tasks to occupy + all of their cores, and if the expected runtime of those tasks is large + enough. + + This is useful for load balancing and adaptivity. + """ if self.total_ncores == 0 or ws.status == "closed": return if occ is None: From 28ce1eda0f6ab4940ce4daa1f309b29a496e6834 Mon Sep 17 00:00:00 2001 From: Benjamin Zaitlen Date: Wed, 22 May 2019 16:06:50 -0400 Subject: [PATCH 21/43] Proxy worker dashboards from scheduler dashboard (#2715) --- dev-requirements.txt | 1 + distributed/bokeh/proxy.py | 130 ++++++++++++++++++ distributed/bokeh/scheduler.py | 35 +++-- distributed/bokeh/scheduler_html.py | 5 +- distributed/bokeh/templates/task.html | 2 +- distributed/bokeh/templates/worker-table.html | 4 +- distributed/bokeh/templates/workers.html | 1 - .../bokeh/tests/test_scheduler_bokeh.py | 44 +++++- .../bokeh/tests/test_scheduler_bokeh_html.py | 7 +- distributed/cli/tests/test_dask_worker.py | 14 +- distributed/scheduler.py | 1 + 11 files changed, 222 insertions(+), 22 deletions(-) create mode 100644 distributed/bokeh/proxy.py diff --git a/dev-requirements.txt b/dev-requirements.txt index 7d684343ca..8cc8f7d256 100644 --- a/dev-requirements.txt +++ b/dev-requirements.txt @@ -10,3 +10,4 @@ jupyter_client >= 4.4.0 ipykernel >= 4.5.2 pytest >= 3.0.5 prometheus_client >= 0.6.0 +jupyter-server-proxy >= 1.1.0 diff --git a/distributed/bokeh/proxy.py b/distributed/bokeh/proxy.py new file mode 100644 index 0000000000..9353e38311 --- /dev/null +++ b/distributed/bokeh/proxy.py @@ -0,0 +1,130 @@ +import logging + +from tornado import web + +logger = logging.getLogger(__name__) + +try: + from jupyter_server_proxy.handlers import ProxyHandler + + class GlobalProxyHandler(ProxyHandler): + """ + A tornado request handler that proxies HTTP and websockets + from a port to any valid endpoint'. + """ + + def initialize(self, server=None, extra=None): + self.scheduler = server + self.extra = extra or {} + + async def http_get(self, port, host, proxied_path): + # route here first + # incoming URI /proxy/{port}/{host}/{proxied_path} + + self.host = host + + # rewrite uri for jupyter-server-proxy handling + uri = "/proxy/%s/%s" % (str(port), proxied_path) + self.request.uri = uri + + # slash is removed during regex in handler + proxied_path = "/%s" % proxied_path + + worker = "%s:%s" % (self.host, str(port)) + if not check_worker_dashboard_exits(self.scheduler, worker): + msg = "Worker <%s> does not exist" % worker + self.set_status(400) + self.finish(msg) + return + return await self.proxy(port, proxied_path) + + async def open(self, port, host, proxied_path): + # finally, proxy to other address/port + return await self.proxy_open(host, port, proxied_path) + + def post(self, port, proxied_path): + return self.proxy(port, proxied_path) + + def put(self, port, proxied_path): + return self.proxy(port, proxied_path) + + def delete(self, port, proxied_path): + return self.proxy(port, proxied_path) + + def head(self, port, proxied_path): + return self.proxy(port, proxied_path) + + def patch(self, port, proxied_path): + return self.proxy(port, proxied_path) + + def options(self, port, proxied_path): + return self.proxy(port, proxied_path) + + def proxy(self, port, proxied_path): + # router here second + # returns ProxyHandler coroutine + return super().proxy(self.host, port, proxied_path) + + +except ImportError: + logger.info( + "To route to workers diagnostics web server " + "please install jupyter-server-proxy: " + "pip install jupyter-server-proxy" + ) + + class GlobalProxyHandler(web.RequestHandler): + """Minimal Proxy handler when jupyter-server-proxy is not installed + """ + + def initialize(self, server=None, extra=None): + self.server = server + self.extra = extra or {} + + def get(self, port, host, proxied_path): + worker_url = "%s:%s/%s" % (host, str(port), proxied_path) + msg = """ +

Try navigating to %s for your worker dashboard

+ +

+ Dask tried to proxy you to that page through your + Scheduler's dashboard connection, but you don't have + jupyter-server-proxy installed. You may want to install it + with either conda or pip, and then restart your scheduler. +

+ +

 conda install jupyter-server-proxy -c conda-forge 

+

 pip install jupyter-server-proxy

+ +

+ The link above should work though if your workers are on a + sufficiently open network. This is common on single machines, + but less common in production clusters. Your IT administrators + will know more +

+ """ % ( + worker_url, + worker_url, + ) + self.write(msg) + + +def check_worker_dashboard_exits(scheduler, worker): + """Check addr:port exists as a worker in scheduler list + + Parameters + ---------- + worker : str + addr:port + + Returns + ------- + bool + """ + addr, port = worker.split(":") + workers = list(scheduler.workers.values()) + for w in workers: + bokeh_port = w.services.get("bokeh", "") + if addr == w.host and port == str(bokeh_port): + return True + return False diff --git a/distributed/bokeh/scheduler.py b/distributed/bokeh/scheduler.py index cce94b356a..e0f5bfffab 100644 --- a/distributed/bokeh/scheduler.py +++ b/distributed/bokeh/scheduler.py @@ -130,7 +130,8 @@ def __init__(self, scheduler, **kwargs): "y": [1, 2], "ms": [1, 2], "color": ["red", "blue"], - "bokeh_address": ["", ""], + "dashboard_port": ["", ""], + "dashboard_host": ["", ""], } ) @@ -152,7 +153,9 @@ def __init__(self, scheduler, **kwargs): # fig.xaxis[0].formatter = NumeralTickFormatter(format='0.0s') fig.x_range.start = 0 - tap = TapTool(callback=OpenURL(url="http://@bokeh_address/")) + tap = TapTool( + callback=OpenURL(url="./proxy/@dashboard_port/@dashboard_host/status") + ) hover = HoverTool() hover.tooltips = "@worker : @occupancy s." @@ -166,10 +169,8 @@ def update(self): with log_errors(): workers = list(self.scheduler.workers.values()) - bokeh_addresses = [] - for ws in workers: - addr = self.scheduler.get_worker_service_addr(ws.address, "bokeh") - bokeh_addresses.append("%s:%d" % addr if addr is not None else "") + dashboard_host = [ws.host for ws in workers] + dashboard_port = [ws.services.get("bokeh", "") for ws in workers] y = list(range(len(workers))) occupancy = [ws.occupancy for ws in workers] @@ -199,7 +200,8 @@ def update(self): "worker": [ws.address for ws in workers], "ms": ms, "color": color, - "bokeh_address": bokeh_addresses, + "dashboard_host": dashboard_host, + "dashboard_port": dashboard_port, "x": x, "y": y, } @@ -317,7 +319,8 @@ def __init__(self, scheduler, width=600, **kwargs): "worker": ["a", "b"], "y": [1, 2], "nbytes-color": ["blue", "blue"], - "bokeh_address": ["", ""], + "dashboard_port": ["", ""], + "dashboard_host": ["", ""], } ) @@ -368,7 +371,11 @@ def __init__(self, scheduler, width=600, **kwargs): fig.yaxis.visible = False fig.ygrid.visible = False - tap = TapTool(callback=OpenURL(url="http://@bokeh_address/")) + tap = TapTool( + callback=OpenURL( + url="./proxy/@dashboard_port/@dashboard_host/status" + ) + ) fig.add_tools(tap) fig.toolbar.logo = None @@ -395,10 +402,8 @@ def update(self): with log_errors(): workers = list(self.scheduler.workers.values()) - bokeh_addresses = [] - for ws in workers: - addr = self.scheduler.get_worker_service_addr(ws.address, "bokeh") - bokeh_addresses.append("%s:%d" % addr if addr is not None else "") + dashboard_host = [ws.host for ws in workers] + dashboard_port = [ws.services.get("bokeh", "") for ws in workers] y = list(range(len(workers))) nprocessing = [len(ws.processing) for ws in workers] @@ -442,7 +447,8 @@ def update(self): "nbytes-half": [nb / 2 for nb in nbytes], "nbytes-color": nbytes_color, "nbytes_text": nbytes_text, - "bokeh_address": bokeh_addresses, + "dashboard_host": dashboard_host, + "dashboard_port": dashboard_port, "worker": [ws.address for ws in workers], "y": y, } @@ -1579,6 +1585,7 @@ def __init__(self, scheduler, io_loop=None, prefix="", **kwargs): self.prefix = prefix self.server_kwargs = kwargs + self.server_kwargs["prefix"] = prefix or None self.apps = { diff --git a/distributed/bokeh/scheduler_html.py b/distributed/bokeh/scheduler_html.py index d1ba2646ed..1d3635c37c 100644 --- a/distributed/bokeh/scheduler_html.py +++ b/distributed/bokeh/scheduler_html.py @@ -7,6 +7,7 @@ from tornado import web from ..utils import log_errors, format_bytes, format_time +from .proxy import GlobalProxyHandler dirname = os.path.dirname(__file__) @@ -42,6 +43,7 @@ def get(self, worker): self.render( "worker.html", title="Worker: " + worker, + scheduler=self.server, Worker=worker, **toolz.merge(self.server.__dict__, ns, self.extra) ) @@ -55,7 +57,7 @@ def get(self, task): "task.html", title="Task: " + task, Task=task, - server=self.server, + scheduler=self.server, **toolz.merge(self.server.__dict__, ns, self.extra) ) @@ -249,6 +251,7 @@ def get(self): (r"individual-plots.json", IndividualPlots), (r"metrics", PrometheusHandler), (r"health", HealthHandler), + (r"proxy/(\d+)/(.*?)/(.*)", GlobalProxyHandler), ] diff --git a/distributed/bokeh/templates/task.html b/distributed/bokeh/templates/task.html index f396a4cba8..8c292da4e4 100644 --- a/distributed/bokeh/templates/task.html +++ b/distributed/bokeh/templates/task.html @@ -122,7 +122,7 @@

Transition Log

Recommended Action - {% for key, start, finish, recommendations, time in server.story(Task) %} + {% for key, start, finish, recommendations, time in scheduler.story(Task) %} {{ fromtimestamp(time) }} {{key}} diff --git a/distributed/bokeh/templates/worker-table.html b/distributed/bokeh/templates/worker-table.html index 90b59c08c5..8a86f8debd 100644 --- a/distributed/bokeh/templates/worker-table.html +++ b/distributed/bokeh/templates/worker-table.html @@ -1,4 +1,4 @@ - +
@@ -20,7 +20,7 @@ {% if 'bokeh' in ws.services %} - + {% else %} {% end %} diff --git a/distributed/bokeh/templates/workers.html b/distributed/bokeh/templates/workers.html index 6a2b7fc934..f300855ac9 100644 --- a/distributed/bokeh/templates/workers.html +++ b/distributed/bokeh/templates/workers.html @@ -5,7 +5,6 @@

Scheduler {{scheduler.address}}

LogsBokeh - {% set worker_list = list(workers.values()) %} {% include "worker-table.html" %} diff --git a/distributed/bokeh/tests/test_scheduler_bokeh.py b/distributed/bokeh/tests/test_scheduler_bokeh.py index f3a57586c7..057aa67965 100644 --- a/distributed/bokeh/tests/test_scheduler_bokeh.py +++ b/distributed/bokeh/tests/test_scheduler_bokeh.py @@ -89,7 +89,7 @@ def test_basic(c, s, a, b): data = ss.source.data assert len(first(data.values())) if component is Occupancy: - assert all(addr.startswith("127.0.0.1:") for addr in data["bokeh_address"]) + assert all(addr == "127.0.0.1" for addr in data["dashboard_host"]) @gen_cluster(client=True) @@ -581,3 +581,45 @@ def test_root_redirect(c, s, a, b): ) assert response.code == 200 assert "/status" in response.effective_url + + +@gen_cluster( + client=True, + scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}, + worker_kwargs={"services": {"bokeh": BokehWorker}}, + timeout=180, +) +def test_proxy_to_workers(c, s, a, b): + try: + import jupyter_server_proxy # noqa: F401 + + proxy_exists = True + except ImportError: + proxy_exists = False + + dashboard_port = s.services["bokeh"].port + http_client = AsyncHTTPClient() + response = yield http_client.fetch("http://localhost:%d/" % dashboard_port) + assert response.code == 200 + assert "/status" in response.effective_url + + for w in [a, b]: + host = w.ip + port = w.service_ports["bokeh"] + proxy_url = "http://localhost:%d/proxy/%s/%s/status" % ( + dashboard_port, + port, + host, + ) + direct_url = "http://localhost:%s/status" % port + http_client = AsyncHTTPClient() + response_proxy = yield http_client.fetch(proxy_url) + response_direct = yield http_client.fetch(direct_url) + + assert response_proxy.code == 200 + if proxy_exists: + assert b"Crossfilter" in response_proxy.body + else: + assert b"pip install jupyter-server-proxy" in response_proxy.body + assert response_direct.code == 200 + assert b"Crossfilter" in response_direct.body diff --git a/distributed/bokeh/tests/test_scheduler_bokeh_html.py b/distributed/bokeh/tests/test_scheduler_bokeh_html.py index 96fe3c2f5d..691121f751 100644 --- a/distributed/bokeh/tests/test_scheduler_bokeh_html.py +++ b/distributed/bokeh/tests/test_scheduler_bokeh_html.py @@ -14,9 +14,14 @@ from dask.sizeof import sizeof from distributed.utils_test import gen_cluster, slowinc, inc from distributed.bokeh.scheduler import BokehScheduler +from distributed.bokeh.worker import BokehWorker -@gen_cluster(client=True, scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}) +@gen_cluster( + client=True, + scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}, + worker_kwargs={"services": {"bokeh": BokehWorker}}, +) def test_connect(c, s, a, b): future = c.submit(lambda x: x + 1, 1) x = c.submit(slowinc, 1, delay=1, retries=5) diff --git a/distributed/cli/tests/test_dask_worker.py b/distributed/cli/tests/test_dask_worker.py index 5ed668e758..aac27061b2 100644 --- a/distributed/cli/tests/test_dask_worker.py +++ b/distributed/cli/tests/test_dask_worker.py @@ -249,8 +249,14 @@ def func(dask_worker): def test_bokeh_non_standard_ports(loop): pytest.importorskip("bokeh") + try: + import jupyter_server_proxy # noqa: F401 - with popen(["dask-scheduler", "--port", "3449", "--no-bokeh"]): + proxy_exists = True + except ImportError: + proxy_exists = False + + with popen(["dask-scheduler", "--port", "3449"]): with popen( ["dask-worker", "tcp://127.0.0.1:3449", "--dashboard-address", ":4833"] ) as proc: @@ -264,9 +270,15 @@ def test_bokeh_non_standard_ports(loop): assert response.ok redirect_resp = requests.get("http://127.0.0.1:4833/main") redirect_resp.ok + # TEST PROXYING WORKS + if proxy_exists: + url = "http://127.0.0.1:8787/proxy/4833/127.0.0.1/status" + response = requests.get(url) + assert response.ok break except Exception: sleep(0.5) assert time() < start + 20 + with pytest.raises(Exception): requests.get("http://localhost:4833/status/") diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 801c0c849d..cce66fb176 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -856,6 +856,7 @@ def __init__( ) self.digests = None self.service_specs = services or {} + self.service_kwargs = service_kwargs or {} self.services = {} self.scheduler_file = scheduler_file worker_ttl = worker_ttl or dask.config.get("distributed.scheduler.worker-ttl") From 6134c754b08b35fd3e98d6128b9cdb2f28bb5300 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Wed, 22 May 2019 15:08:48 -0500 Subject: [PATCH 22/43] Replace register_worker_callbacks with worker plugins (#2453) * Add worker plugins * add docstring * Replace legacy worker_callbacks with worker_plugins * add and test name keyword * fix missing import * black * respond to feedback * Handle errors again * Expand docstring --- distributed/client.py | 92 ++++++++++++++++++++---- distributed/scheduler.py | 17 +++-- distributed/tests/test_worker.py | 7 +- distributed/tests/test_worker_plugins.py | 68 ++++++++++++++++++ distributed/worker.py | 57 ++++++++++++--- 5 files changed, 206 insertions(+), 35 deletions(-) create mode 100644 distributed/tests/test_worker_plugins.py diff --git a/distributed/client.py b/distributed/client.py index 6b5bc2811b..6b22f5b8ee 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -89,6 +89,7 @@ parse_timedelta, shutting_down, Any, + has_keyword, ) from .versions import get_versions @@ -3854,17 +3855,6 @@ def _get_task_stream( else: raise gen.Return(msgs) - @gen.coroutine - def _register_worker_callbacks(self, setup=None): - responses = yield self.scheduler.register_worker_callbacks(setup=dumps(setup)) - results = {} - for key, resp in responses.items(): - if resp["status"] == "OK": - results[key] = resp["result"] - elif resp["status"] == "error": - six.reraise(*clean_exception(**resp)) - raise gen.Return(results) - def register_worker_callbacks(self, setup=None): """ Registers a setup callback function for all current and future workers. @@ -3883,7 +3873,85 @@ def register_worker_callbacks(self, setup=None): setup : callable(dask_worker: Worker) -> None Function to register and run on all workers """ - return self.sync(self._register_worker_callbacks, setup=setup) + return self.register_worker_plugin(_WorkerSetupPlugin(setup)) + + @gen.coroutine + def _register_worker_plugin(self, plugin=None, name=None): + responses = yield self.scheduler.register_worker_plugin( + plugin=dumps(plugin), name=name + ) + for response in responses.values(): + if response["status"] == "error": + exc = response["exception"] + typ = type(exc) + tb = response["traceback"] + six.reraise(typ, exc, tb) + raise gen.Return(responses) + + def register_worker_plugin(self, plugin=None, name=None): + """ + Registers a lifecycle worker plugin for all current and future workers. + + This registers a new object to handle setup and teardown for workers in + this cluster. The plugin will instantiate itself on all currently + connected workers. It will also be run on any worker that connects in + the future. + + The plugin should be an object with ``setup`` and ``teardown`` methods. + It must be serializable with the pickle or cloudpickle modules. + + If the plugin has a ``name`` attribute, or if the ``name=`` keyword is + used then that will control idempotency. A a plugin with that name has + already registered then any future plugins will not run. + + For alternatives to plugins, you may also wish to look into preload + scripts. + + Parameters + ---------- + plugin: object + The plugin object to pass to the workers + name: str, optional + A name for the plugin. + Registering a plugin with the same name will have no effect. + + Examples + -------- + >>> class MyPlugin: + ... def __init__(self, *args, **kwargs): + ... pass # the constructor is up to you + ... def setup(self, worker: dask.distributed.Worker): + ... pass + ... def teardown(self, worker: dask.distributed.Worker): + ... pass + + >>> plugin = MyPlugin(1, 2, 3) + >>> client.register_worker_plugin(plugin) + + You can get access to the plugin with the ``get_worker`` function + + >>> client.register_worker_plugin(other_plugin, name='my-plugin') + >>> def f(): + ... worker = get_worker() + ... plugin = worker.plugins['my-plugin'] + ... return plugin.my_state + + >>> future = client.run(f) + """ + return self.sync(self._register_worker_plugin, plugin=plugin, name=name) + + +class _WorkerSetupPlugin(object): + """ This is used to support older setup functions as callbacks """ + + def __init__(self, setup): + self._setup = setup + + def setup(self, worker): + if has_keyword(self._setup, "dask_worker"): + return self._setup(dask_worker=worker) + else: + return self._setup() class Executor(Client): diff --git a/distributed/scheduler.py b/distributed/scheduler.py index cce66fb176..57d768f95f 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1003,7 +1003,7 @@ def __init__( self.log = deque( maxlen=dask.config.get("distributed.scheduler.transition-log-length") ) - self.worker_setups = [] + self.worker_plugins = [] worker_handlers = { "task-finished": self.handle_task_finished, @@ -1062,7 +1062,7 @@ def __init__( "heartbeat_worker": self.heartbeat_worker, "get_task_status": self.get_task_status, "get_task_stream": self.get_task_stream, - "register_worker_callbacks": self.register_worker_callbacks, + "register_worker_plugin": self.register_worker_plugin, } self._transitions = { @@ -1510,7 +1510,7 @@ def add_worker( "status": "OK", "time": time(), "heartbeat-interval": heartbeat_interval(len(self.workers)), - "worker-setups": self.worker_setups, + "worker-plugins": self.worker_plugins, } ) yield self.handle_worker(comm=comm, worker=address) @@ -3407,14 +3407,13 @@ def get_task_stream(self, comm=None, start=None, stop=None, count=None): return ts.collect(start=start, stop=stop, count=count) @gen.coroutine - def register_worker_callbacks(self, comm, setup=None): + def register_worker_plugin(self, comm, plugin, name=None): """ Registers a setup function, and call it on every worker """ - if setup is None: - raise gen.Return({}) - - self.worker_setups.append(setup) + self.worker_plugins.append(plugin) - responses = yield self.broadcast(msg=dict(op="run", function=setup)) + responses = yield self.broadcast( + msg=dict(op="plugin-add", plugin=plugin, name=name) + ) raise gen.Return(responses) ##################### diff --git a/distributed/tests/test_worker.py b/distributed/tests/test_worker.py index 4541e183e4..9fc967eef5 100644 --- a/distributed/tests/test_worker.py +++ b/distributed/tests/test_worker.py @@ -1312,7 +1312,6 @@ def test_startup2(): return os.getenv("MY_ENV_VALUE", None) == "WORKER_ENV_VALUE" # Nothing has been run yet - assert len(s.worker_setups) == 0 result = yield c.run(test_import) assert list(result.values()) == [False] * 2 result = yield c.run(test_startup2) @@ -1327,7 +1326,6 @@ def test_startup2(): # Add a preload function response = yield c.register_worker_callbacks(setup=mystartup) assert len(response) == 2 - assert len(s.worker_setups) == 1 # Check it has been ran on existing worker result = yield c.run(test_import) @@ -1342,7 +1340,6 @@ def test_startup2(): # Register another preload function response = yield c.register_worker_callbacks(setup=mystartup2) assert len(response) == 2 - assert len(s.worker_setups) == 2 # Check it has been run result = yield c.run(test_startup2) @@ -1356,7 +1353,9 @@ def test_startup2(): assert list(result.values()) == [True] yield worker.close() - # Final exception test + +@gen_cluster(client=True) +def test_register_worker_callbacks_err(c, s, a, b): with pytest.raises(ZeroDivisionError): yield c.register_worker_callbacks(setup=lambda: 1 / 0) diff --git a/distributed/tests/test_worker_plugins.py b/distributed/tests/test_worker_plugins.py new file mode 100644 index 0000000000..2538845978 --- /dev/null +++ b/distributed/tests/test_worker_plugins.py @@ -0,0 +1,68 @@ +from distributed.utils_test import gen_cluster +from distributed import Worker + + +class MyPlugin: + name = "MyPlugin" + + def __init__(self, data): + self.data = data + + def setup(self, worker): + assert isinstance(worker, Worker) + self.worker = worker + self.worker._my_plugin_status = "setup" + self.worker._my_plugin_data = self.data + + def teardown(self, worker): + assert isinstance(worker, Worker) + self.worker._my_plugin_status = "teardown" + + +@gen_cluster(client=True, ncores=[]) +def test_create_with_client(c, s): + yield c.register_worker_plugin(MyPlugin(123)) + + worker = Worker(s.address, loop=s.loop) + yield worker._start() + assert worker._my_plugin_status == "setup" + assert worker._my_plugin_data == 123 + + yield worker._close() + assert worker._my_plugin_status == "teardown" + + +@gen_cluster(client=True, worker_kwargs={"plugins": [MyPlugin(5)]}) +def test_create_on_construction(c, s, a, b): + assert len(a.plugins) == len(b.plugins) == 1 + assert a._my_plugin_status == "setup" + assert a._my_plugin_data == 5 + + +@gen_cluster(client=True, worker_kwargs={"plugins": [MyPlugin(5)]}) +def test_idempotence_with_name(c, s, a, b): + a._my_plugin_data = 100 + + yield c.register_worker_plugin(MyPlugin(5)) + + assert a._my_plugin_data == 100 # call above has no effect + + +@gen_cluster(client=True, worker_kwargs={"plugins": [MyPlugin(5)]}) +def test_duplicate_with_no_name(c, s, a, b): + assert len(a.plugins) == len(b.plugins) == 1 + + plugin = MyPlugin(10) + plugin.name = "other-name" + + yield c.register_worker_plugin(plugin) + + assert len(a.plugins) == len(b.plugins) == 2 + + assert a._my_plugin_data == 10 + + yield c.register_worker_plugin(plugin) + assert len(a.plugins) == len(b.plugins) == 2 + + yield c.register_worker_plugin(plugin, name="foo") + assert len(a.plugins) == len(b.plugins) == 3 diff --git a/distributed/worker.py b/distributed/worker.py index f418939309..3fcc477bf4 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -10,6 +10,7 @@ import random import threading import sys +import uuid import warnings import weakref import psutil @@ -307,6 +308,7 @@ def __init__( protocol=None, dashboard_address=None, nanny=None, + plugins=(), low_level_profiler=dask.config.get("distributed.worker.profile.low-level"), **kwargs ): @@ -576,6 +578,7 @@ def __init__( "versions": self.versions, "actor_execute": self.actor_execute, "actor_attribute": self.actor_attribute, + "plugin-add": self.plugin_add, } stream_handlers = { @@ -638,6 +641,9 @@ def __init__( ) self.periodic_callbacks["profile-cycle"] = pc + self.plugins = {} + self._pending_plugins = plugins + Worker._instances.add(self) ################## @@ -763,16 +769,9 @@ def _register_with_scheduler(self): if response["status"] != "OK": raise ValueError("Unexpected response from register: %r" % (response,)) else: - # Retrieve eventual init functions and run them - for function_bytes in response["worker-setups"]: - setup_function = pickle.loads(function_bytes) - if has_arg(setup_function, "dask_worker"): - result = setup_function(dask_worker=self) - else: - result = setup_function() - logger.info( - "Init function %s ran: output=%s" % (setup_function, result) - ) + yield [ + self.plugin_add(plugin=plugin) for plugin in response["worker-plugins"] + ] logger.info(" Registered to: %26s", self.scheduler.address) logger.info("-" * 49) @@ -968,6 +967,9 @@ def _start(self, addr_or_port=0): setproctitle("dask-worker [%s]" % self.address) + yield [self.plugin_add(plugin=plugin) for plugin in self._pending_plugins] + self._pending_plugins = () + yield self._register_with_scheduler() self.start_periodic_callbacks() @@ -998,6 +1000,12 @@ def close(self, report=True, timeout=10, nanny=True, executor_wait=True): self.status = "closing" setproctitle("dask-worker [closing]") + yield [ + plugin.teardown(self) + for plugin in self.plugins.values() + if hasattr(plugin, "teardown") + ] + self.stop() for pc in self.periodic_callbacks.values(): pc.stop() @@ -2206,6 +2214,35 @@ def run(self, comm, function, args=(), wait=True, kwargs=None): def run_coroutine(self, comm, function, args=(), kwargs=None, wait=True): return run(self, comm, function=function, args=args, kwargs=kwargs, wait=wait) + @gen.coroutine + def plugin_add(self, comm=None, plugin=None, name=None): + with log_errors(pdb=False): + if isinstance(plugin, bytes): + plugin = pickle.loads(plugin) + if not name: + if hasattr(plugin, "name"): + name = plugin.name + else: + name = funcname(plugin) + "-" + str(uuid.uuid4()) + + assert name + + if name in self.plugins: + return {"status": "repeat"} + else: + self.plugins[name] = plugin + + logger.info("Starting Worker plugin %s" % name) + try: + result = plugin.setup(worker=self) + if isinstance(result, gen.Future): + result = yield result + except Exception as e: + msg = error_message(e) + return msg + else: + return {"status": "OK"} + @gen.coroutine def actor_execute(self, comm=None, actor=None, function=None, args=(), kwargs={}): separate_thread = kwargs.pop("separate_thread", True) From 6e0c0a6b90b1d3c3f686f0c968e9cf3d0c354413 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Wed, 22 May 2019 16:45:53 -0500 Subject: [PATCH 23/43] Add SpecificationCluster (#2675) This is intended to be a base for LocalCluster (and others) that want to specify more heterogeneous information about workers. This forces the use of Python 3 and introduces more asyncio and async def handling. This cleans up a number of intermittent testing failures and improves our testing harness hygeine. --- .../setup_conda_environment.cmd | 2 +- continuous_integration/travis/install.sh | 2 +- distributed/__init__.py | 2 +- distributed/cli/dask_worker.py | 2 +- distributed/client.py | 24 +- distributed/comm/tcp.py | 1 + distributed/core.py | 8 +- distributed/deploy/__init__.py | 1 + distributed/deploy/adaptive.py | 4 +- distributed/deploy/cluster.py | 32 +- distributed/deploy/local.py | 326 ++---------------- distributed/deploy/spec.py | 297 ++++++++++++++++ distributed/deploy/tests/py3_test_deploy.py | 15 +- distributed/deploy/tests/test_adaptive.py | 56 ++- distributed/deploy/tests/test_local.py | 120 ++++--- distributed/deploy/tests/test_spec_cluster.py | 115 ++++++ distributed/deploy/utils_test.py | 17 +- distributed/nanny.py | 34 +- distributed/scheduler.py | 12 +- distributed/tests/test_as_completed.py | 4 +- distributed/tests/test_asyncprocess.py | 1 + distributed/tests/test_client.py | 39 ++- distributed/tests/test_nanny.py | 9 +- distributed/tests/test_scheduler.py | 16 +- distributed/tests/test_worker.py | 31 +- distributed/tests/test_worker_client.py | 4 +- distributed/utils.py | 2 + distributed/utils_test.py | 84 ++--- distributed/worker.py | 15 +- 29 files changed, 774 insertions(+), 501 deletions(-) create mode 100644 distributed/deploy/spec.py create mode 100644 distributed/deploy/tests/test_spec_cluster.py diff --git a/continuous_integration/setup_conda_environment.cmd b/continuous_integration/setup_conda_environment.cmd index cd201ff46d..5748a8cf20 100644 --- a/continuous_integration/setup_conda_environment.cmd +++ b/continuous_integration/setup_conda_environment.cmd @@ -50,7 +50,7 @@ call activate %CONDA_ENV% %PIP_INSTALL% git+https://github.com/joblib/joblib.git --upgrade %PIP_INSTALL% git+https://github.com/dask/zict --upgrade -%PIP_INSTALL% pytest-repeat pytest-timeout pytest-faulthandler sortedcollections +%PIP_INSTALL% pytest-repeat pytest-timeout pytest-faulthandler sortedcollections pytest-asyncio @rem Display final environment (for reproducing) %CONDA% list diff --git a/continuous_integration/travis/install.sh b/continuous_integration/travis/install.sh index bba69dd3ac..f1ff25a9bf 100644 --- a/continuous_integration/travis/install.sh +++ b/continuous_integration/travis/install.sh @@ -59,7 +59,7 @@ conda install -q \ conda install -c defaults -c conda-forge libunwind conda install --no-deps -c defaults -c numba -c conda-forge stacktrace -pip install -q pytest-repeat pytest-faulthandler +pip install -q pytest-repeat pytest-faulthandler pytest-asyncio pip install -q git+https://github.com/dask/dask.git --upgrade --no-deps pip install -q git+https://github.com/joblib/joblib.git --upgrade --no-deps diff --git a/distributed/__init__.py b/distributed/__init__.py index 7b2bc4ab08..2a632607cf 100644 --- a/distributed/__init__.py +++ b/distributed/__init__.py @@ -4,7 +4,7 @@ from dask.config import config from .actor import Actor, ActorFuture from .core import connect, rpc -from .deploy import LocalCluster, Adaptive +from .deploy import LocalCluster, Adaptive, SpecCluster from .diagnostics import progress from .client import ( Client, diff --git a/distributed/cli/dask_worker.py b/distributed/cli/dask_worker.py index e383095b38..439bdaf4a6 100755 --- a/distributed/cli/dask_worker.py +++ b/distributed/cli/dask_worker.py @@ -114,7 +114,7 @@ @click.option( "--name", type=str, - default="", + default=None, help="A unique name for this worker like 'worker-1'. " "If used with --nprocs then the process number " "will be appended like name-0, name-1, name-2, ...", diff --git a/distributed/client.py b/distributed/client.py index 6b22f5b8ee..afe6f6ef39 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -2,8 +2,8 @@ import atexit from collections import defaultdict -from concurrent.futures import ThreadPoolExecutor -from concurrent.futures._base import DoneAndNotDoneFutures, CancelledError +from concurrent.futures import ThreadPoolExecutor, CancelledError +from concurrent.futures._base import DoneAndNotDoneFutures from contextlib import contextmanager import copy from datetime import timedelta @@ -44,6 +44,8 @@ from tornado.ioloop import IOLoop from tornado.queues import Queue +from asyncio import iscoroutine + from .batched import BatchedSend from .utils_comm import ( WrappedKey, @@ -1309,7 +1311,13 @@ def close(self, timeout=no_default): if self._start_arg is None: with ignoring(AttributeError): - self.cluster.close() + f = self.cluster.close() + if iscoroutine(f): + + async def _(): + await f + + self.sync(_) sync(self.loop, self._close, fast=True) @@ -1644,10 +1652,11 @@ def wait(k): st = self.futures[key] exception = st.exception traceback = st.traceback - except (AttributeError, KeyError): - six.reraise(CancelledError, CancelledError(key), None) + except (KeyError, AttributeError): + exc = CancelledError(key) else: six.reraise(type(exception), exception, traceback) + raise exc if errors == "skip": bad_keys.add(key) bad_data[key] = None @@ -4134,7 +4143,10 @@ def _track_future(self, future): except CancelledError: pass if self.with_results: - result = yield future._result(raiseit=False) + try: + result = yield future._result(raiseit=False) + except CancelledError as exc: + result = exc with self.lock: self.futures[future] -= 1 if not self.futures[future]: diff --git a/distributed/comm/tcp.py b/distributed/comm/tcp.py index 85dbe2ce27..d5351c7d56 100644 --- a/distributed/comm/tcp.py +++ b/distributed/comm/tcp.py @@ -431,6 +431,7 @@ def start(self): break else: raise exc + self.get_host_port() # trigger assignment to self.bound_address def stop(self): tcp_server, self.tcp_server = self.tcp_server, None diff --git a/distributed/core.py b/distributed/core.py index 9b1d408a03..17685c9d2d 100644 --- a/distributed/core.py +++ b/distributed/core.py @@ -489,14 +489,16 @@ def handle_stream(self, comm, extra=None, every_cycle=[]): @gen.coroutine def close(self): - self.listener.stop() + for pc in self.periodic_callbacks.values(): + pc.stop() + if self.listener: + self.listener.stop() for i in range(20): # let comms close naturally for a second if not self._comms: break else: yield gen.sleep(0.05) - for comm in self._comms: - comm.close() + yield [comm.close() for comm in self._comms] for cb in self._ongoing_coroutines: cb.cancel() for i in range(10): diff --git a/distributed/deploy/__init__.py b/distributed/deploy/__init__.py index 35abf0a643..9b5e478c30 100644 --- a/distributed/deploy/__init__.py +++ b/distributed/deploy/__init__.py @@ -4,6 +4,7 @@ from .cluster import Cluster from .local import LocalCluster +from .spec import SpecCluster from .adaptive import Adaptive with ignoring(ImportError): diff --git a/distributed/deploy/adaptive.py b/distributed/deploy/adaptive.py index 8c26060963..793e80d984 100644 --- a/distributed/deploy/adaptive.py +++ b/distributed/deploy/adaptive.py @@ -272,7 +272,7 @@ def _retire_workers(self, workers=None): logger.info("Retiring workers %s", workers) f = self.cluster.scale_down(workers) - if gen.is_future(f): + if hasattr(f, "__await__"): yield f raise gen.Return(workers) @@ -354,7 +354,7 @@ def _adapt(self): if status == "up": f = self.cluster.scale_up(**recommendations) self.log.append((time(), "up", recommendations)) - if gen.is_future(f): + if hasattr(f, "__await__"): yield f elif status == "down": diff --git a/distributed/deploy/cluster.py b/distributed/deploy/cluster.py index f170d4ea5a..8425b836a4 100644 --- a/distributed/deploy/cluster.py +++ b/distributed/deploy/cluster.py @@ -1,12 +1,23 @@ +from datetime import timedelta import logging import os from weakref import ref import dask +from tornado import gen from .adaptive import Adaptive -from ..utils import format_bytes, PeriodicCallback, log_errors, ignoring +from ..compatibility import get_thread_identity +from ..utils import ( + format_bytes, + PeriodicCallback, + log_errors, + ignoring, + sync, + thread_state, +) + logger = logging.getLogger(__name__) @@ -215,3 +226,22 @@ def update(): def _ipython_display_(self, **kwargs): return self._widget()._ipython_display_(**kwargs) + + @property + def asynchronous(self): + return ( + self._asynchronous + or getattr(thread_state, "asynchronous", False) + or hasattr(self.loop, "_thread_identity") + and self.loop._thread_identity == get_thread_identity() + ) + + def sync(self, func, *args, **kwargs): + if kwargs.pop("asynchronous", None) or self.asynchronous: + callback_timeout = kwargs.pop("callback_timeout", None) + future = func(*args, **kwargs) + if callback_timeout is not None: + future = gen.with_timeout(timedelta(seconds=callback_timeout), future) + return future + else: + return sync(self.loop, func, *args, **kwargs) diff --git a/distributed/deploy/local.py b/distributed/deploy/local.py index 832e8f3e05..17150fdf70 100644 --- a/distributed/deploy/local.py +++ b/distributed/deploy/local.py @@ -1,29 +1,14 @@ from __future__ import print_function, division, absolute_import import atexit -from datetime import timedelta import logging import math import warnings import weakref -import toolz from dask.utils import factors -from tornado import gen - -from .cluster import Cluster -from ..compatibility import get_thread_identity -from ..core import CommClosedError -from ..utils import ( - sync, - ignoring, - All, - silence_logging, - LoopRunner, - log_errors, - thread_state, - parse_timedelta, -) + +from .spec import SpecCluster from ..nanny import Nanny from ..scheduler import Scheduler from ..worker import Worker, parse_memory_limit, _ncores @@ -31,7 +16,7 @@ logger = logging.getLogger(__name__) -class LocalCluster(Cluster): +class LocalCluster(SpecCluster): """ Create local Scheduler and Workers This creates a "cluster" of a scheduler and workers running on the local @@ -105,8 +90,8 @@ def __init__( processes=True, loop=None, start=None, - ip=None, host=None, + ip=None, scheduler_port=0, silence_logs=logging.WARN, dashboard_address=":8787", @@ -127,15 +112,6 @@ def __init__( warnings.warn("The ip keyword has been moved to host") host = ip - if start is not None: - msg = ( - "The start= parameter is deprecated. " - "LocalCluster always starts. " - "For asynchronous operation use the following: \n\n" - " cluster = yield LocalCluster(asynchronous=True)" - ) - raise ValueError(msg) - if diagnostics_port is not None: warnings.warn( "diagnostics_port has been deprecated. " @@ -161,12 +137,8 @@ def __init__( if host is None and not protocol.startswith("inproc") and not interface: host = "127.0.0.1" - self.silence_logs = silence_logs - self._asynchronous = asynchronous services = services or {} worker_services = worker_services or {} - if silence_logs: - self._old_logging_level = silence_logging(level=silence_logs) if n_workers is None and threads_per_worker is None: if processes: n_workers, threads_per_worker = nprocesses_nthreads(_ncores) @@ -188,268 +160,42 @@ def __init__( "dashboard_address": worker_dashboard_address, "interface": interface, "protocol": protocol, + "security": security, + "silence_logs": silence_logs, } ) - self._loop_runner = LoopRunner(loop=loop, asynchronous=asynchronous) - self.loop = self._loop_runner.loop - - self.scheduler = Scheduler( - loop=self.loop, - host=host, - services=services, - service_kwargs=service_kwargs, - security=security, - port=scheduler_port, - interface=interface, - protocol=protocol, - dashboard_address=dashboard_address, - blocked_handlers=blocked_handlers, + scheduler = { + "cls": Scheduler, + "options": dict( + host=host, + services=services, + service_kwargs=service_kwargs, + security=security, + port=scheduler_port, + interface=interface, + protocol=protocol, + dashboard_address=dashboard_address, + blocked_handlers=blocked_handlers, + ), + } + + worker = { + "cls": worker_class or (Worker if not processes else Nanny), + "options": worker_kwargs, + } + + workers = {i: worker for i in range(n_workers)} + + super(LocalCluster, self).__init__( + scheduler=scheduler, + workers=workers, + worker=worker, + loop=loop, + asynchronous=asynchronous, + silence_logs=silence_logs, ) - - self.workers = [] - self.worker_kwargs = worker_kwargs - if security: - self.worker_kwargs["security"] = security - - if not worker_class: - worker_class = Worker if not processes else Nanny - self.worker_class = worker_class - - self.start(n_workers=n_workers) - - clusters_to_close.add(self) - - def __repr__(self): - return "LocalCluster(%r, workers=%d, ncores=%d)" % ( - self.scheduler_address, - len(self.workers), - sum(w.ncores for w in self.workers), - ) - - def __await__(self): - return self._started.__await__() - - @property - def asynchronous(self): - return ( - self._asynchronous - or getattr(thread_state, "asynchronous", False) - or hasattr(self.loop, "_thread_identity") - and self.loop._thread_identity == get_thread_identity() - ) - - def sync(self, func, *args, **kwargs): - if kwargs.pop("asynchronous", None) or self.asynchronous: - callback_timeout = kwargs.pop("callback_timeout", None) - future = func(*args, **kwargs) - if callback_timeout is not None: - future = gen.with_timeout(timedelta(seconds=callback_timeout), future) - return future - else: - return sync(self.loop, func, *args, **kwargs) - - def start(self, **kwargs): - self._loop_runner.start() - if self._asynchronous: - self._started = self._start(**kwargs) - else: - self.sync(self._start, **kwargs) - - @gen.coroutine - def _start(self, n_workers=0): - """ - Start all cluster services. - """ - if self.status == "running": - return - - self.scheduler.start() - - yield [self._start_worker(**self.worker_kwargs) for i in range(n_workers)] - yield self.scheduler - - self.status = "running" - - raise gen.Return(self) - - @gen.coroutine - def _start_worker(self, death_timeout=60, **kwargs): - if self.status and self.status.startswith("clos"): - warnings.warn( - "Tried to start a worker while status=='%s'" % self.status, stacklevel=2 - ) - return - - if self.processes: - kwargs["quiet"] = True - - w = yield self.worker_class( - self.scheduler.address, - loop=self.loop, - death_timeout=death_timeout, - silence_logs=self.silence_logs, - **kwargs - ) - - self.workers.append(w) - - while w.status != "closed" and w.worker_address not in self.scheduler.workers: - yield gen.sleep(0.01) - - if w.status == "closed" and self.scheduler.status == "running": - self.workers.remove(w) - raise gen.TimeoutError("Worker failed to start") - - raise gen.Return(w) - - def start_worker(self, **kwargs): - """ Add a new worker to the running cluster - - Parameters - ---------- - port: int (optional) - Port on which to serve the worker, defaults to 0 or random - ncores: int (optional) - Number of threads to use. Defaults to number of logical cores - - Examples - -------- - >>> c = LocalCluster() # doctest: +SKIP - >>> c.start_worker(ncores=2) # doctest: +SKIP - - Returns - ------- - The created Worker or Nanny object. Can be discarded. - """ - return self.sync(self._start_worker, **kwargs) - - @gen.coroutine - def _stop_worker(self, w): - yield w.close() - if w in self.workers: - self.workers.remove(w) - - def stop_worker(self, w): - """ Stop a running worker - - Examples - -------- - >>> c = LocalCluster() # doctest: +SKIP - >>> w = c.start_worker(ncores=2) # doctest: +SKIP - >>> c.stop_worker(w) # doctest: +SKIP - """ - self.sync(self._stop_worker, w) - - @gen.coroutine - def _close(self, timeout="2s"): - # Can be 'closing' as we're called by close() below - if self.status == "closed": - return - self.status = "closing" - - with ignoring(gen.TimeoutError, CommClosedError, OSError): - yield gen.with_timeout( - timedelta(seconds=parse_timedelta(timeout)), - self.scheduler.close(close_workers=True), - ) - - with ignoring(gen.TimeoutError): - yield gen.with_timeout( - timedelta(seconds=parse_timedelta(timeout)), - All([self._stop_worker(w) for w in self.workers]), - ) - del self.workers[:] - self.status = "closed" - - def close(self, timeout=20): - """ Close the cluster """ - if self.status == "closed": - return - - try: - result = self.sync(self._close, callback_timeout=timeout) - except RuntimeError: # IOLoop is closed - result = None - - if hasattr(self, "_old_logging_level"): - if self.asynchronous: - result.add_done_callback( - lambda _: silence_logging(self._old_logging_level) - ) - else: - silence_logging(self._old_logging_level) - - if not self.asynchronous: - self._loop_runner.stop() - - return result - - @gen.coroutine - def scale_up(self, n, **kwargs): - """ Bring the total count of workers up to ``n`` - - This function/coroutine should bring the total number of workers up to - the number ``n``. - - This can be implemented either as a function or as a Tornado coroutine. - """ - with log_errors(): - kwargs2 = toolz.merge(self.worker_kwargs, kwargs) - yield [ - self._start_worker(**kwargs2) - for i in range(n - len(self.scheduler.workers)) - ] - - # clean up any closed worker - self.workers = [w for w in self.workers if w.status != "closed"] - - @gen.coroutine - def scale_down(self, workers): - """ Remove ``workers`` from the cluster - - Given a list of worker addresses this function should remove those - workers from the cluster. This may require tracking which jobs are - associated to which worker address. - - This can be implemented either as a function or as a Tornado coroutine. - """ - with log_errors(): - # clean up any closed worker - self.workers = [w for w in self.workers if w.status != "closed"] - workers = set(workers) - - # we might be given addresses - if all(isinstance(w, str) for w in workers): - workers = {w for w in self.workers if w.worker_address in workers} - - # stop the provided workers - yield [self._stop_worker(w) for w in workers] - - def __del__(self): - self.close() - - def __enter__(self): - return self - - def __exit__(self, *args): - self.close() - - @gen.coroutine - def __aenter__(self): - yield self._started - raise gen.Return(self) - - @gen.coroutine - def __aexit__(self, typ, value, traceback): - yield self._close() - - @property - def scheduler_address(self): - try: - return self.scheduler.address - except ValueError: - return "" + self.scale(n_workers) def nprocesses_nthreads(n): diff --git a/distributed/deploy/spec.py b/distributed/deploy/spec.py new file mode 100644 index 0000000000..9a4385e505 --- /dev/null +++ b/distributed/deploy/spec.py @@ -0,0 +1,297 @@ +import asyncio +import weakref + +from tornado import gen + +from .cluster import Cluster +from ..utils import LoopRunner, silence_logging, ignoring +from ..scheduler import Scheduler + + +class SpecCluster(Cluster): + """ Cluster that requires a full specification of workers + + The SpecCluster class expects a full specification of the Scheduler and + Workers to use. It removes any handling of user inputs (like threads vs + processes, number of cores, and so on) and any handling of cluster resource + managers (like pods, jobs, and so on). Instead, it expects this + information to be passed in scheduler and worker specifications. This + class does handle all of the logic around asynchronously cleanly setting up + and tearing things down at the right times. Hopefully it can form a base + for other more user-centric classes. + + Parameters + ---------- + workers: dict + A dictionary mapping names to worker classes and their specifications + See example below + scheduler: dict, optional + A similar mapping for a scheduler + worker: dict + A specification of a single worker. + This is used for any new workers that are created. + asynchronous: bool + If this is intended to be used directly within an event loop with + async/await + silence_logs: bool + Whether or not we should silence logging when setting up the cluster. + + Examples + -------- + To create a SpecCluster you specify how to set up a Scheduler and Workers + + >>> from dask.distributed import Scheduler, Worker, Nanny + >>> scheduler = {'cls': Scheduler, 'options': {"dashboard_address": ':8787'}} + >>> workers = { + ... 'my-worker': {"cls": Worker, "options": {"ncores": 1}}, + ... 'my-nanny': {"cls": Nanny, "options": {"ncores": 2}}, + ... } + >>> cluster = SpecCluster(scheduler=scheduler, workers=workers) + + The worker spec is stored as the ``.worker_spec`` attribute + + >>> cluster.worker_spec + { + 'my-worker': {"cls": Worker, "options": {"ncores": 1}}, + 'my-nanny': {"cls": Nanny, "options": {"ncores": 2}}, + } + + While the instantiation of this spec is stored in the ``.workers`` + attribute + + >>> cluster.workers + { + 'my-worker': + 'my-nanny': + } + + Should the spec change, we can await the cluster or call the + ``._correct_state`` method to align the actual state to the specified + state. + + We can also ``.scale(...)`` the cluster, which adds new workers of a given + form. + + >>> worker = {'cls': Worker, 'options': {}} + >>> cluster = SpecCluster(scheduler=scheduler, worker=worker) + >>> cluster.worker_spec + {} + + >>> cluster.scale(3) + >>> cluster.worker_spec + { + 0: {'cls': Worker, 'options': {}}, + 1: {'cls': Worker, 'options': {}}, + 2: {'cls': Worker, 'options': {}}, + } + + Note that above we are using the standard ``Worker`` and ``Nanny`` classes, + however in practice other classes could be used that handle resource + management like ``KubernetesPod`` or ``SLURMJob``. The spec does not need + to conform to the expectations of the standard Dask Worker class. It just + needs to be called with the provided options, support ``__await__`` and + ``close`` methods and the ``worker_address`` property.. + + Also note that uniformity of the specification is not required. Other API + could be added externally (in subclasses) that adds workers of different + specifications into the same dictionary. + """ + + def __init__( + self, + workers=None, + scheduler=None, + worker=None, + asynchronous=False, + loop=None, + silence_logs=False, + ): + self._created = weakref.WeakSet() + if scheduler is None: + try: + from distributed.bokeh.scheduler import BokehScheduler + except ImportError: + services = {} + else: + services = {("bokeh", 8787): BokehScheduler} + scheduler = {"cls": Scheduler, "options": {"services": services}} + + self.scheduler_spec = scheduler + self.worker_spec = workers or {} + self.new_spec = worker + self.workers = {} + self._i = 0 + self._asynchronous = asynchronous + + if silence_logs: + self._old_logging_level = silence_logging(level=silence_logs) + + self._loop_runner = LoopRunner(loop=loop, asynchronous=asynchronous) + self.loop = self._loop_runner.loop + + self.scheduler = self.scheduler_spec["cls"]( + loop=self.loop, **self.scheduler_spec["options"] + ) + self.status = "created" + self._correct_state_waiting = None + + if not self.asynchronous: + self._loop_runner.start() + self.sync(self._start) + self.sync(self._correct_state) + self.sync(self._wait_for_workers) + + async def _start(self): + while self.status == "starting": + await asyncio.sleep(0.01) + if self.status == "running": + return + if self.status == "closed": + raise ValueError("Cluster is closed") + + self._lock = asyncio.Lock() + self.status = "starting" + self.scheduler = await self.scheduler + self.status = "running" + + def _correct_state(self): + if self._correct_state_waiting: + # If people call this frequently, we only want to run it once + return self._correct_state_waiting + else: + task = asyncio.ensure_future(self._correct_state_internal()) + self._correct_state_waiting = task + return task + + async def _correct_state_internal(self): + async with self._lock: + self._correct_state_waiting = None + + pre = list(set(self.workers)) + to_close = set(self.workers) - set(self.worker_spec) + if to_close: + await self.scheduler.retire_workers(workers=list(to_close)) + tasks = [self.workers[w].close() for w in to_close] + await asyncio.wait(tasks) + for task in tasks: # for tornado gen.coroutine support + await task + for name in to_close: + del self.workers[name] + + to_open = set(self.worker_spec) - set(self.workers) + workers = [] + for name in to_open: + d = self.worker_spec[name] + cls, opts = d["cls"], d.get("options", {}) + if "name" not in opts: + opts = opts.copy() + opts["name"] = name + worker = cls(self.scheduler.address, **opts) + self._created.add(worker) + workers.append(worker) + if workers: + await asyncio.wait(workers) + for w in workers: + w._cluster = weakref.ref(self) + await w # for tornado gen.coroutine support + self.workers.update(dict(zip(to_open, workers))) + + def __await__(self): + async def _(): + if self.status == "created": + await self._start() + await self.scheduler + await self._correct_state() + if self.workers: + await asyncio.wait(list(self.workers.values())) # maybe there are more + await self._wait_for_workers() + return self + + return _().__await__() + + async def _wait_for_workers(self): + # TODO: this function needs to query scheduler and worker state + # remotely without assuming that they are local + while {d["name"] for d in self.scheduler.identity()["workers"].values()} != set( + self.workers + ): + if ( + any(w.status == "closed" for w in self.workers.values()) + and self.scheduler.status == "running" + ): + raise gen.TimeoutError("Worker unexpectedly closed") + await asyncio.sleep(0.1) + + async def __aenter__(self): + await self + return self + + async def __aexit__(self, typ, value, traceback): + await self.close() + + async def _close(self): + while self.status == "closing": + await asyncio.sleep(0.1) + if self.status == "closed": + return + self.status = "closing" + + async with self._lock: + await self.scheduler.close(close_workers=True) + self.scale(0) + await self._correct_state() + for w in self._created: + assert w.status == "closed" + + if hasattr(self, "_old_logging_level"): + silence_logging(self._old_logging_level) + + self.status = "closed" + + def close(self): + with ignoring(RuntimeError): # loop closed during process shutdown + return self.sync(self._close) + + def __del__(self): + if self.status != "closed": + self.close() + + def __enter__(self): + self.sync(self._correct_state) + self.sync(self._wait_for_workers) + assert self.status == "running" + return self + + def __exit__(self, typ, value, traceback): + self.close() + self._loop_runner.stop() + + def scale(self, n): + while len(self.worker_spec) > n: + self.worker_spec.popitem() + + while len(self.worker_spec) < n: + while self._i in self.worker_spec: + self._i += 1 + self.worker_spec[self._i] = self.new_spec + + self.loop.add_callback(self._correct_state) + + async def scale_down(self, workers): + workers = set(workers) + + # TODO: this is linear cost. We should be indexing by name or something + to_close = [w for w in self.workers.values() if w.address in workers] + for k, v in self.workers.items(): + if v.worker_address in workers: + del self.worker_spec[k] + + await self + + scale_up = scale # backwards compatibility + + def __repr__(self): + return "SpecCluster(%r, workers=%d)" % ( + self.scheduler_address, + len(self.workers), + ) diff --git a/distributed/deploy/tests/py3_test_deploy.py b/distributed/deploy/tests/py3_test_deploy.py index 4c8fb2f86d..7a66ecf942 100644 --- a/distributed/deploy/tests/py3_test_deploy.py +++ b/distributed/deploy/tests/py3_test_deploy.py @@ -1,14 +1,13 @@ from distributed import LocalCluster from distributed.utils_test import loop # noqa: F401 +import pytest -def test_async_with(loop): - async def f(): - async with LocalCluster(processes=False, asynchronous=True) as cluster: - w = cluster.workers - assert w +@pytest.mark.asyncio +async def test_async_with(): + async with LocalCluster(processes=False, asynchronous=True) as cluster: + w = cluster.workers + assert w - assert not w - - loop.run_sync(f) + assert not w diff --git a/distributed/deploy/tests/test_adaptive.py b/distributed/deploy/tests/test_adaptive.py index 50c4f0a45a..8915c72135 100644 --- a/distributed/deploy/tests/test_adaptive.py +++ b/distributed/deploy/tests/test_adaptive.py @@ -2,12 +2,13 @@ from time import sleep +import pytest from toolz import frequencies, pluck from tornado import gen from tornado.ioloop import IOLoop -from distributed import Client, wait, Adaptive, LocalCluster -from distributed.utils_test import gen_cluster, gen_test, slowinc, inc +from distributed import Client, wait, Adaptive, LocalCluster, SpecCluster, Worker +from distributed.utils_test import gen_cluster, gen_test, slowinc, inc, clean from distributed.utils_test import loop, nodebug # noqa: F401 from distributed.metrics import time @@ -162,19 +163,17 @@ def scale_down(self, workers): assert len(s.workers) == 2 +@pytest.mark.xfail(reason="need to rework adaptive") @gen_test(timeout=30) def test_min_max(): - loop = IOLoop.current() cluster = yield LocalCluster( 0, scheduler_port=0, silence_logs=False, processes=False, dashboard_address=None, - loop=loop, asynchronous=True, ) - yield cluster._start() try: adapt = Adaptive( cluster.scheduler, @@ -184,7 +183,7 @@ def test_min_max(): interval="20 ms", wait_count=10, ) - c = yield Client(cluster, asynchronous=True, loop=loop) + c = yield Client(cluster, asynchronous=True) start = time() while not cluster.scheduler.workers: @@ -359,17 +358,18 @@ def test_no_more_workers_than_tasks(): def test_basic_no_loop(): - try: - with LocalCluster( - 0, scheduler_port=0, silence_logs=False, dashboard_address=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: - loop.add_callback(loop.stop) + with clean(threads=False): + try: + with LocalCluster( + 0, scheduler_port=0, silence_logs=False, dashboard_address=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: + loop.add_callback(loop.stop) @gen_test(timeout=None) @@ -408,25 +408,17 @@ def test_target_duration(): @gen_test(timeout=None) def test_worker_keys(): """ Ensure that redefining adapt with a lower maximum removes workers """ - cluster = yield LocalCluster( - 0, + cluster = yield SpecCluster( + workers={ + "a-1": {"cls": Worker}, + "a-2": {"cls": Worker}, + "b-1": {"cls": Worker}, + "b-2": {"cls": Worker}, + }, asynchronous=True, - processes=False, - scheduler_port=0, - silence_logs=False, - dashboard_address=None, ) try: - yield [ - cluster.start_worker(name="a-1"), - cluster.start_worker(name="a-2"), - cluster.start_worker(name="b-1"), - cluster.start_worker(name="b-2"), - ] - - while len(cluster.scheduler.workers) != 4: - yield gen.sleep(0.01) def key(ws): return ws.name.split("-")[0] diff --git a/distributed/deploy/tests/test_local.py b/distributed/deploy/tests/test_local.py index ed9e3bb2db..4498611d7e 100644 --- a/distributed/deploy/tests/test_local.py +++ b/distributed/deploy/tests/test_local.py @@ -17,6 +17,7 @@ from distributed.deploy.local import LocalCluster, nprocesses_nthreads from distributed.metrics import time from distributed.utils_test import ( + clean, inc, gen_test, slowinc, @@ -46,7 +47,7 @@ def test_simple(loop): x = e.submit(inc, 1) x.result() assert x.key in c.scheduler.tasks - assert any(w.data == {x.key: 2} for w in c.workers) + assert any(w.data == {x.key: 2} for w in c.workers.values()) assert e.loop is c.loop @@ -87,10 +88,10 @@ def test_procs(): silence_logs=False, ) as c: assert len(c.workers) == 2 - assert all(isinstance(w, Worker) for w in c.workers) + assert all(isinstance(w, Worker) for w in c.workers.values()) with Client(c.scheduler.address) as e: - assert all(w.ncores == 3 for w in c.workers) - assert all(isinstance(w, Worker) for w in c.workers) + assert all(w.ncores == 3 for w in c.workers.values()) + assert all(isinstance(w, Worker) for w in c.workers.values()) repr(c) with LocalCluster( @@ -102,12 +103,12 @@ def test_procs(): silence_logs=False, ) as c: assert len(c.workers) == 2 - assert all(isinstance(w, Nanny) for w in c.workers) + assert all(isinstance(w, Nanny) for w in c.workers.values()) with Client(c.scheduler.address) as e: assert all(v == 3 for v in e.ncores().values()) - c.start_worker() - assert all(isinstance(w, Nanny) for w in c.workers) + c.scale(3) + assert all(isinstance(w, Nanny) for w in c.workers.values()) repr(c) @@ -171,7 +172,7 @@ def test_transports_tcp_port(): @pytest.mark.skipif("sys.version_info[0] == 2", reason="") class LocalTest(ClusterTest, unittest.TestCase): Cluster = partial(LocalCluster, silence_logs=False, dashboard_address=None) - kwargs = {"dashboard_address": None} + kwargs = {"dashboard_address": None, "processes": False} @pytest.mark.skipif("sys.version_info[0] == 2", reason="") @@ -208,12 +209,13 @@ def test_duplicate_clients(): for msg in info.list ) yield c1.close() + yield c2.close() def test_Client_kwargs(loop): with Client(loop=loop, processes=False, n_workers=2, silence_logs=False) as c: assert len(c.cluster.workers) == 2 - assert all(isinstance(w, Worker) for w in c.cluster.workers) + assert all(isinstance(w, Worker) for w in c.cluster.workers.values()) assert c.cluster.status == "closed" @@ -230,14 +232,14 @@ def test_defaults(): with LocalCluster( scheduler_port=0, silence_logs=False, dashboard_address=None ) as c: - assert sum(w.ncores for w in c.workers) == _ncores - assert all(isinstance(w, Nanny) for w in c.workers) + assert sum(w.ncores for w in c.workers.values()) == _ncores + assert all(isinstance(w, Nanny) for w in c.workers.values()) with LocalCluster( processes=False, scheduler_port=0, silence_logs=False, dashboard_address=None ) as c: - assert sum(w.ncores for w in c.workers) == _ncores - assert all(isinstance(w, Worker) for w in c.workers) + assert sum(w.ncores for w in c.workers.values()) == _ncores + assert all(isinstance(w, Worker) for w in c.workers.values()) assert len(c.workers) == 1 with LocalCluster( @@ -248,7 +250,7 @@ def test_defaults(): else: # n_workers not a divisor of _ncores => threads are overcommitted expected_total_threads = max(2, _ncores + 1) - assert sum(w.ncores for w in c.workers) == expected_total_threads + assert sum(w.ncores for w in c.workers.values()) == expected_total_threads with LocalCluster( threads_per_worker=_ncores * 2, @@ -264,7 +266,7 @@ def test_defaults(): silence_logs=False, dashboard_address=None, ) as c: - assert all(w.ncores == 1 for w in c.workers) + assert all(w.ncores == 1 for w in c.workers.values()) with LocalCluster( threads_per_worker=2, n_workers=3, @@ -273,18 +275,19 @@ def test_defaults(): dashboard_address=None, ) as c: assert len(c.workers) == 3 - assert all(w.ncores == 2 for w in c.workers) + assert all(w.ncores == 2 for w in c.workers.values()) def test_worker_params(): with LocalCluster( + processes=False, n_workers=2, scheduler_port=0, silence_logs=False, dashboard_address=None, memory_limit=500, ) as c: - assert [w.memory_limit for w in c.workers] == [500] * 2 + assert [w.memory_limit for w in c.workers.values()] == [500] * 2 def test_memory_limit_none(): @@ -302,24 +305,28 @@ def test_memory_limit_none(): def test_cleanup(): - c = LocalCluster(2, scheduler_port=0, silence_logs=False, dashboard_address=None) - port = c.scheduler.port - c.close() - c2 = LocalCluster( - 2, scheduler_port=port, silence_logs=False, dashboard_address=None - ) - c.close() + with clean(threads=False): + c = LocalCluster( + 2, scheduler_port=0, silence_logs=False, dashboard_address=None + ) + port = c.scheduler.port + c.close() + c2 = LocalCluster( + 2, scheduler_port=port, silence_logs=False, dashboard_address=None + ) + c2.close() def test_repeated(): - with LocalCluster( - 0, scheduler_port=8448, silence_logs=False, dashboard_address=None - ) as c: - pass - with LocalCluster( - 0, scheduler_port=8448, silence_logs=False, dashboard_address=None - ) as c: - pass + with clean(threads=False): + with LocalCluster( + 0, scheduler_port=8448, silence_logs=False, dashboard_address=None + ) as c: + pass + with LocalCluster( + 0, scheduler_port=8448, silence_logs=False, dashboard_address=None + ) as c: + pass @pytest.mark.parametrize("processes", [True, False]) @@ -373,15 +380,15 @@ def test_scale_up_and_down(): assert not cluster.workers - yield cluster.scale_up(2) + cluster.scale(2) + yield cluster assert len(cluster.workers) == 2 assert len(cluster.scheduler.ncores) == 2 - addr = cluster.workers[0].address - yield cluster.scale_down([addr]) + cluster.scale(1) + yield cluster assert len(cluster.workers) == 1 - assert addr not in cluster.scheduler.ncores yield c.close() yield cluster.close() @@ -437,7 +444,7 @@ def test_memory(loop, n_workers): dashboard_address=None, loop=loop, ) as cluster: - assert sum(w.memory_limit for w in cluster.workers) <= TOTAL_MEMORY + assert sum(w.memory_limit for w in cluster.workers.values()) <= TOTAL_MEMORY @pytest.mark.parametrize("n_workers", [None, 3]) @@ -486,11 +493,13 @@ def test_bokeh_kwargs(loop): def test_io_loop_periodic_callbacks(loop): - with LocalCluster(loop=loop, silence_logs=False) as cluster: + with LocalCluster( + loop=loop, port=0, dashboard_address=None, silence_logs=False + ) as cluster: assert cluster.scheduler.loop is loop for pc in cluster.scheduler.periodic_callbacks.values(): assert pc.io_loop is loop - for worker in cluster.workers: + for worker in cluster.workers.values(): for pc in worker.periodic_callbacks.values(): assert pc.io_loop is loop @@ -772,7 +781,7 @@ def test_worker_class_worker(loop): scheduler_port=0, dashboard_address=None, ) as cluster: - assert all(isinstance(w, MyWorker) for w in cluster.workers) + assert all(isinstance(w, MyWorker) for w in cluster.workers.values()) def test_worker_class_nanny(loop): @@ -786,8 +795,37 @@ class MyNanny(Nanny): scheduler_port=0, dashboard_address=None, ) as cluster: - assert all(isinstance(w, MyNanny) for w in cluster.workers) + assert all(isinstance(w, MyNanny) for w in cluster.workers.values()) + + +@pytest.mark.asyncio +async def test_worker_class_nanny_async(): + class MyNanny(Nanny): + pass + + async with LocalCluster( + n_workers=2, + worker_class=MyNanny, + scheduler_port=0, + dashboard_address=None, + asynchronous=True, + ) as cluster: + assert all(isinstance(w, MyNanny) for w in cluster.workers.values()) if sys.version_info >= (3, 5): from distributed.deploy.tests.py3_test_deploy import * # noqa F401 + + +def test_starts_up_sync(loop): + cluster = LocalCluster( + n_workers=2, + loop=loop, + processes=False, + scheduler_port=0, + dashboard_address=None, + ) + try: + assert len(cluster.scheduler.workers) == 2 + finally: + cluster.close() diff --git a/distributed/deploy/tests/test_spec_cluster.py b/distributed/deploy/tests/test_spec_cluster.py new file mode 100644 index 0000000000..cfc1242727 --- /dev/null +++ b/distributed/deploy/tests/test_spec_cluster.py @@ -0,0 +1,115 @@ +from dask.distributed import SpecCluster, Worker, Client, Scheduler +from distributed.utils_test import loop # noqa: F401 +import pytest + + +class MyWorker(Worker): + pass + + +class BrokenWorker(Worker): + def __await__(self): + async def _(): + raise Exception("Worker Broken") + + return _().__await__() + + +worker_spec = { + 0: {"cls": Worker, "options": {"ncores": 1}}, + 1: {"cls": Worker, "options": {"ncores": 2}}, + "my-worker": {"cls": MyWorker, "options": {"ncores": 3}}, +} +scheduler = {"cls": Scheduler, "options": {"port": 0}} + + +@pytest.mark.asyncio +async def test_specification(): + async with SpecCluster( + workers=worker_spec, scheduler=scheduler, asynchronous=True + ) as cluster: + assert cluster.worker_spec is worker_spec + + assert len(cluster.workers) == 3 + assert set(cluster.workers) == set(worker_spec) + assert isinstance(cluster.workers[0], Worker) + assert isinstance(cluster.workers[1], Worker) + assert isinstance(cluster.workers["my-worker"], MyWorker) + + assert cluster.workers[0].ncores == 1 + assert cluster.workers[1].ncores == 2 + assert cluster.workers["my-worker"].ncores == 3 + + async with Client(cluster, asynchronous=True) as client: + result = await client.submit(lambda x: x + 1, 10) + assert result == 11 + + for name in cluster.workers: + assert cluster.workers[name].name == name + + +def test_spec_sync(loop): + worker_spec = { + 0: {"cls": Worker, "options": {"ncores": 1}}, + 1: {"cls": Worker, "options": {"ncores": 2}}, + "my-worker": {"cls": MyWorker, "options": {"ncores": 3}}, + } + with SpecCluster(workers=worker_spec, scheduler=scheduler, loop=loop) as cluster: + assert cluster.worker_spec is worker_spec + + assert len(cluster.workers) == 3 + assert set(cluster.workers) == set(worker_spec) + assert isinstance(cluster.workers[0], Worker) + assert isinstance(cluster.workers[1], Worker) + assert isinstance(cluster.workers["my-worker"], MyWorker) + + assert cluster.workers[0].ncores == 1 + assert cluster.workers[1].ncores == 2 + assert cluster.workers["my-worker"].ncores == 3 + + with Client(cluster, loop=loop) as client: + assert cluster.loop is cluster.scheduler.loop + assert cluster.loop is client.loop + result = client.submit(lambda x: x + 1, 10).result() + assert result == 11 + + +def test_loop_started(): + cluster = SpecCluster(worker_spec) + + +@pytest.mark.asyncio +async def test_scale(): + worker = {"cls": Worker, "options": {"ncores": 1}} + async with SpecCluster( + asynchronous=True, scheduler=scheduler, worker=worker + ) as cluster: + assert not cluster.workers + assert not cluster.worker_spec + + # Scale up + cluster.scale(2) + assert not cluster.workers + assert cluster.worker_spec + + await cluster + assert len(cluster.workers) == 2 + + # Scale down + cluster.scale(1) + assert len(cluster.workers) == 2 + + await cluster + assert len(cluster.workers) == 1 + + +@pytest.mark.asyncio +async def test_broken_worker(): + with pytest.raises(Exception) as info: + async with SpecCluster( + asynchronous=True, + workers={"good": {"cls": Worker}, "bad": {"cls": BrokenWorker}}, + ) as cluster: + pass + + assert "Broken" in str(info.value) diff --git a/distributed/deploy/utils_test.py b/distributed/deploy/utils_test.py index 9bc8caccca..9da8d64cd5 100644 --- a/distributed/deploy/utils_test.py +++ b/distributed/deploy/utils_test.py @@ -1,5 +1,7 @@ from ..client import Client +import pytest + class ClusterTest(object): Cluster = None @@ -13,26 +15,15 @@ def tearDown(self): self.client.close() self.cluster.close() + @pytest.mark.xfail() def test_cores(self): + info = self.client.scheduler_info() assert len(self.client.ncores()) == 2 def test_submit(self): future = self.client.submit(lambda x: x + 1, 1) assert future.result() == 2 - def test_start_worker(self): - a = self.client.ncores() - w = self.cluster.start_worker(ncores=3) - b = self.client.ncores() - - assert len(b) == 1 + len(a) - assert any(v == 3 for v in b.values()) - - self.cluster.stop_worker(w) - - c = self.client.ncores() - assert c == a - def test_context_manager(self): with self.Cluster(**self.kwargs) as c: with Client(c) as e: diff --git a/distributed/nanny.py b/distributed/nanny.py index 842ec765d7..a27f713ea6 100644 --- a/distributed/nanny.py +++ b/distributed/nanny.py @@ -18,7 +18,7 @@ from .comm import get_address_host, get_local_address_for, unparse_host_port from .comm.addressing import address_from_user_args -from .core import rpc, RPCClosed, CommClosedError, coerce_to_address +from .core import RPCClosed, CommClosedError, coerce_to_address from .metrics import time from .node import ServerNode from .process import AsyncProcess @@ -30,6 +30,7 @@ silence_logging, json_load_robust, PeriodicCallback, + parse_timedelta, ) from .worker import _ncores, run, parse_memory_limit, Worker @@ -78,6 +79,11 @@ def __init__( protocol=None, **worker_kwargs ): + self.loop = loop or IOLoop.current() + self.security = security or Security() + assert isinstance(self.security, Security) + self.connection_args = self.security.get_connection_args("worker") + self.listen_args = self.security.get_listen_args("worker") if scheduler_file: cfg = json_load_robust(scheduler_file) @@ -88,12 +94,13 @@ def __init__( self.scheduler_addr = coerce_to_address(scheduler_ip) else: self.scheduler_addr = coerce_to_address((scheduler_ip, scheduler_port)) + self._given_worker_port = worker_port self.ncores = ncores or _ncores self.reconnect = reconnect self.validate = validate self.resources = resources - self.death_timeout = death_timeout + self.death_timeout = parse_timedelta(death_timeout) self.preload = preload self.preload_argv = preload_argv self.Worker = Worker if worker_class is None else worker_class @@ -105,15 +112,8 @@ def __init__( "distributed.worker.memory.terminate" ) - self.security = security or Security() - assert isinstance(self.security, Security) - self.connection_args = self.security.get_connection_args("worker") - self.listen_args = self.security.get_listen_args("worker") - self.local_dir = local_dir - self.loop = loop or IOLoop.current() - self.scheduler = rpc(self.scheduler_addr, connection_args=self.connection_args) self.services = services self.name = name self.quiet = quiet @@ -135,9 +135,11 @@ def __init__( } super(Nanny, self).__init__( - handlers, io_loop=self.loop, connection_args=self.connection_args + handlers=handlers, io_loop=self.loop, connection_args=self.connection_args ) + self.scheduler = self.rpc(self.scheduler_addr) + if self.memory_limit: pc = PeriodicCallback(self.memory_monitor, 100, io_loop=self.loop) self.periodic_callbacks["memory"] = pc @@ -240,7 +242,6 @@ def kill(self, comm=None, timeout=2): deadline = self.loop.time() + timeout yield self.process.kill(timeout=0.8 * (deadline - self.loop.time())) - yield self._unregister(deadline - self.loop.time()) @gen.coroutine def instantiate(self, comm=None): @@ -376,8 +377,12 @@ def close(self, comm=None, timeout=5, report=None): """ Close the worker process, stop all comms. """ - if self.status in ("closing", "closed"): + while self.status == "closing": + yield gen.sleep(0.01) + + if self.status == "closed": raise gen.Return("OK") + self.status = "closing" logger.info("Closing Nanny at %r", self.address) self.stop() @@ -388,9 +393,10 @@ def close(self, comm=None, timeout=5, report=None): pass self.process = None self.rpc.close() - self.scheduler.close_rpc() self.status = "closed" - raise gen.Return("OK") + if comm: + yield comm.write("OK") + yield ServerNode.close(self) class WorkerProcess(object): diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 57d768f95f..991ff1a210 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1365,7 +1365,10 @@ def heartbeat_worker( self.host_info[host]["last-seen"] = local_now frac = 1 / 20 / len(self.workers) - self.bandwidth = self.bandwidth * (1 - frac) + metrics["bandwidth"] * frac + try: + self.bandwidth = self.bandwidth * (1 - frac) + metrics["bandwidth"] * frac + except KeyError: + pass ws = self.workers.get(address) if not ws: @@ -1990,7 +1993,10 @@ def cancel_key(self, key, client, retries=5, force=False): """ Cancel a particular key and all dependents """ # TODO: this should be converted to use the transition mechanism ts = self.tasks.get(key) - cs = self.clients[client] + try: + cs = self.clients[client] + except KeyError: + return if ts is None or not ts.who_wants: # no key yet, lets try again in a moment if retries: self.loop.add_future( @@ -3085,7 +3091,7 @@ def retire_workers( except KeyError: # keys left during replicate pass - workers = {self.workers[w] for w in workers} + workers = {self.workers[w] for w in workers if w in self.workers} if len(workers) > 0: # Keys orphaned by retiring those workers keys = set.union(*[w.has_what for w in workers]) diff --git a/distributed/tests/test_as_completed.py b/distributed/tests/test_as_completed.py index 8e66b58dd4..a584025ad0 100644 --- a/distributed/tests/test_as_completed.py +++ b/distributed/tests/test_as_completed.py @@ -1,4 +1,4 @@ -from concurrent.futures._base import CancelledError +from concurrent.futures import CancelledError from operator import add import random from time import sleep @@ -226,7 +226,7 @@ def test_as_completed_with_results_no_raise(client): assert y.status == "cancelled" assert z.status == "finished" - assert isinstance(dd[y][0], CancelledError) + assert isinstance(dd[y][0], CancelledError) or dd[y][0] == 6 assert isinstance(dd[x][0][1], RuntimeError) assert dd[z][0] == 2 diff --git a/distributed/tests/test_asyncprocess.py b/distributed/tests/test_asyncprocess.py index 1e7a5d2804..3cb3eee14d 100644 --- a/distributed/tests/test_asyncprocess.py +++ b/distributed/tests/test_asyncprocess.py @@ -49,6 +49,7 @@ def threads_info(q): q.put(threading.current_thread().name) +@pytest.mark.xfail(reason="Intermittent failure") @nodebug @gen_test() def test_simple(): diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 4cd196fa2f..28c2f939eb 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -22,7 +22,6 @@ import pytest from toolz import identity, isdistinct, concat, pluck, valmap, partial, first, merge from tornado import gen -from tornado.ioloop import IOLoop import dask from dask import delayed @@ -3321,7 +3320,12 @@ def test_get_foo_lost_keys(c, s, u, v, w): @pytest.mark.slow -@gen_cluster(client=True, Worker=Nanny, check_new_threads=False) +@gen_cluster( + client=True, + Worker=Nanny, + check_new_threads=False, + worker_kwargs={"death_timeout": "500ms"}, +) def test_bad_tasks_fail(c, s, a, b): f = c.submit(sys.exit, 1) with pytest.raises(KilledWorker) as info: @@ -3575,24 +3579,29 @@ def test_reconnect_timeout(c, s): @pytest.mark.skipif( sys.version_info[0] == 2, reason="Semaphore.acquire doesn't support timeout option" ) -@pytest.mark.xfail(reason="TODO: intermittent failures") +# @pytest.mark.xfail(reason="TODO: intermittent failures") @pytest.mark.parametrize("worker,count,repeat", [(Worker, 100, 5), (Nanny, 10, 20)]) def test_open_close_many_workers(loop, worker, count, repeat): psutil = pytest.importorskip("psutil") proc = psutil.Process() - with cluster(nworkers=0, active_rpc_timeout=20) as (s, _): + with cluster(nworkers=0, active_rpc_timeout=2) as (s, _): gc.collect() before = proc.num_fds() done = Semaphore(0) running = weakref.WeakKeyDictionary() + workers = set() + status = True @gen.coroutine def start_worker(sleep, duration, repeat=1): for i in range(repeat): yield gen.sleep(sleep) + if not status: + return w = worker(s["address"], loop=loop) running[w] = None + workers.add(w) yield w addr = w.worker_address running[w] = addr @@ -3621,6 +3630,12 @@ def start_worker(sleep, duration, repeat=1): sleep(0.2) assert time() < start + 10 + status = False + + [c.sync(w.close) for w in list(workers)] + for w in workers: + assert w.status == "closed" + start = time() while proc.num_fds() > before: print("fds:", before, proc.num_fds()) @@ -4232,23 +4247,23 @@ def test_scatter_dict_workers(c, s, a, b): @pytest.mark.slow @gen_test() def test_client_timeout(): - loop = IOLoop.current() c = Client("127.0.0.1:57484", asynchronous=True) - s = Scheduler(loop=loop) + s = Scheduler(loop=c.loop, port=57484) yield gen.sleep(4) try: - s.start(("127.0.0.1", 57484)) + yield s except EnvironmentError: # port in use + yield c.close() return start = time() - while not c.scheduler_comm: - yield gen.sleep(0.1) + yield c + try: assert time() < start + 2 - - yield c.close() - yield s.close() + finally: + yield c.close() + yield s.close() @gen_cluster(client=True) diff --git a/distributed/tests/test_nanny.py b/distributed/tests/test_nanny.py index 4c18b5242a..be0a05afc2 100644 --- a/distributed/tests/test_nanny.py +++ b/distributed/tests/test_nanny.py @@ -110,10 +110,6 @@ def test_nanny_process_failure(c, s): s.stop() -def test_nanny_no_port(): - _ = str(Nanny("127.0.0.1", 8786)) - - @gen_cluster(ncores=[]) def test_run(s): pytest.importorskip("psutil") @@ -319,12 +315,13 @@ def test_scheduler_address_config(c, s): @pytest.mark.slow -@gen_test() +@gen_test(timeout=20) def test_wait_for_scheduler(): with captured_logger("distributed") as log: w = Nanny("127.0.0.1:44737") - w._start() + w.start() yield gen.sleep(6) + yield w.close() log = log.getvalue() assert "error" not in log.lower(), log diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 9f61e5e710..f5ce276b8d 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -822,11 +822,12 @@ def test_file_descriptors(c, s): assert num_fds_6 < num_fds_5 + N yield [n.close() for n in nannies] + yield c.close() assert not s.rpc.open - assert not any( - occ for addr, occ in c.rpc.occupied.items() if occ != s.address - ), list(c.rpc._created) + for addr, occ in c.rpc.occupied.items(): + for comm in occ: + assert comm.closed() or comm.peer_address != s.address, comm assert not s.stream_comms start = time() @@ -1141,7 +1142,8 @@ def test_scheduler_file(): assert data["address"] == s.address c = yield Client(scheduler_file=fn, loop=s.loop, asynchronous=True) - yield s.close() + yield c.close() + yield s.close() @pytest.mark.xfail(reason="") @@ -1555,7 +1557,7 @@ def test_close_workers(s, a, b): ) @gen_test() def test_host_address(): - s = yield Scheduler(host="127.0.0.2") + s = yield Scheduler(host="127.0.0.2", port=0) assert "127.0.0.2" in s.address yield s.close() @@ -1563,10 +1565,10 @@ def test_host_address(): @gen_test() def test_dashboard_address(): pytest.importorskip("bokeh") - s = yield Scheduler(dashboard_address="127.0.0.1:8901") + s = yield Scheduler(dashboard_address="127.0.0.1:8901", port=0) assert s.services["bokeh"].port == 8901 yield s.close() - s = yield Scheduler(dashboard_address="127.0.0.1") + s = yield Scheduler(dashboard_address="127.0.0.1", port=0) assert s.services["bokeh"].port yield s.close() diff --git a/distributed/tests/test_worker.py b/distributed/tests/test_worker.py index 9fc967eef5..bf4e483f44 100644 --- a/distributed/tests/test_worker.py +++ b/distributed/tests/test_worker.py @@ -1,6 +1,7 @@ from __future__ import print_function, division, absolute_import from concurrent.futures import ThreadPoolExecutor +from datetime import timedelta import logging from numbers import Number from operator import add @@ -312,15 +313,20 @@ def test_worker_with_port_zero(): @pytest.mark.slow -def test_worker_waits_for_center_to_come_up(loop): +def test_worker_waits_for_scheduler(loop): @gen.coroutine def f(): - w = yield Worker("127.0.0.1", 8007) + w = Worker("127.0.0.1", 8007) + try: + yield gen.with_timeout(timedelta(seconds=3), w) + except TimeoutError: + pass + else: + assert False + assert w.status not in ("closed", "running") + yield w.close(timeout=0.1) - try: - loop.run_sync(f, timeout=4) - except TimeoutError: - pass + loop.run_sync(f) @gen_cluster(client=True, ncores=[("127.0.0.1", 1)]) @@ -355,12 +361,13 @@ def test_gather(s, a, b): assert a.data["y"] == b.data["y"] -def test_io_loop(loop): - s = Scheduler(loop=loop) - s.listen(0) - assert s.io_loop is loop - w = Worker(s.address, loop=loop) - assert w.io_loop is loop +@pytest.mark.asyncio +async def test_io_loop(): + s = await Scheduler(port=0) + w = await Worker(s.address, loop=s.loop) + assert w.io_loop is s.loop + await s.close() + await w.close() @gen_cluster(client=True, ncores=[]) diff --git a/distributed/tests/test_worker_client.py b/distributed/tests/test_worker_client.py index 2d4632b0b5..9c4616e9d2 100644 --- a/distributed/tests/test_worker_client.py +++ b/distributed/tests/test_worker_client.py @@ -202,7 +202,9 @@ def f(x): b = db.from_sequence([1, 2]) b2 = b.map(f) - with Client(loop=loop, processes=False, set_as_default=True) as c: + with Client( + loop=loop, processes=False, set_as_default=True, dashboard_address=None + ) as c: assert dask.base.get_scheduler() == c.get for i in range(2): b2.compute() diff --git a/distributed/utils.py b/distributed/utils.py index 466a96fdfd..55508a4c57 100644 --- a/distributed/utils.py +++ b/distributed/utils.py @@ -1242,6 +1242,8 @@ def parse_timedelta(s, default="seconds"): >>> parse_timedelta(timedelta(seconds=3)) # also supports timedeltas 3 """ + if s is None: + return None if isinstance(s, timedelta): return s.total_seconds() if isinstance(s, Number): diff --git a/distributed/utils_test.py b/distributed/utils_test.py index c44f417747..d61046f2a4 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -116,40 +116,35 @@ def cleanup_global_workers(): @pytest.fixture def loop(): - Worker._instances.clear() - _global_clients.clear() - with pristine_loop() as loop: - # Monkey-patch IOLoop.start to wait for loop stop - orig_start = loop.start - is_stopped = threading.Event() - is_stopped.set() + with check_instances(): + with pristine_loop() as loop: + # Monkey-patch IOLoop.start to wait for loop stop + orig_start = loop.start + is_stopped = threading.Event() + is_stopped.set() - def start(): - is_stopped.clear() - try: - orig_start() - finally: - is_stopped.set() + def start(): + is_stopped.clear() + try: + orig_start() + finally: + is_stopped.set() - loop.start = start + loop.start = start - yield loop + yield loop - # Stop the loop in case it's still running - try: - sync(loop, cleanup_global_workers, callback_timeout=0.500) - loop.add_callback(loop.stop) - except RuntimeError as e: - if not re.match("IOLoop is clos(ed|ing)", str(e)): - raise - except gen.TimeoutError: - pass - else: - is_stopped.wait() - Worker._instances.clear() - - _cleanup_dangling() - _global_clients.clear() + # Stop the loop in case it's still running + try: + sync(loop, cleanup_global_workers, callback_timeout=0.500) + loop.add_callback(loop.stop) + except RuntimeError as e: + if not re.match("IOLoop is clos(ed|ing)", str(e)): + raise + except gen.TimeoutError: + pass + else: + is_stopped.wait() @pytest.fixture @@ -464,13 +459,13 @@ def background_read(): raise gen.Return(msg) -def run_scheduler(q, nputs, **kwargs): +def run_scheduler(q, nputs, port=0, **kwargs): from distributed import Scheduler # On Python 2.7 and Unix, fork() is used to spawn child processes, # so avoid inheriting the parent's IO loop. with pristine_loop() as loop: - scheduler = Scheduler(validate=True, host="127.0.0.1", **kwargs) + scheduler = Scheduler(validate=True, host="127.0.0.1", port=port, **kwargs) done = scheduler.start() for i in range(nputs): @@ -735,9 +730,9 @@ def cluster( client.close() start = time() - while list(ws): - sleep(0.01) - assert time() < start + 1, "Workers still around after one second" + while len(ws): + sleep(0.1) + assert time() < start + 3, ("Workers still around after two seconds", list(ws)) @gen.coroutine @@ -769,15 +764,12 @@ def test_foo(): def _(func): def test_func(): - with pristine_loop() as loop: + with clean() as loop: if iscoroutinefunction(func): cor = func else: cor = gen.coroutine(func) - try: - loop.run_sync(cor, timeout=timeout) - finally: - loop.stop() + loop.run_sync(cor, timeout=timeout) return test_func @@ -798,7 +790,9 @@ def start_cluster( scheduler_kwargs={}, worker_kwargs={}, ): - s = Scheduler(loop=loop, validate=True, security=security, **scheduler_kwargs) + s = Scheduler( + loop=loop, validate=True, security=security, port=0, **scheduler_kwargs + ) done = s.start(scheduler_addr) workers = [ Worker( @@ -1483,6 +1477,9 @@ def check_instances(): Client._instances.clear() Worker._instances.clear() Scheduler._instances.clear() + # assert all(n.status == "closed" for n in Nanny._instances), { + # n: n.status for n in Nanny._instances + # } Nanny._instances.clear() _global_clients.clear() Comm._instances.clear() @@ -1513,6 +1510,11 @@ def check_instances(): print("Unclosed Comms", L) # raise ValueError("Unclosed Comms", L) + assert all(n.status == "closed" or n.status == "init" for n in Nanny._instances), { + n: n.status for n in Nanny._instances + } + + Nanny._instances.clear() DequeHandler.clear_all_instances() diff --git a/distributed/worker.py b/distributed/worker.py index 3fcc477bf4..667bd83490 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -436,7 +436,7 @@ def __init__( self.ncores = ncores or _ncores self.total_resources = resources or {} self.available_resources = (resources or {}).copy() - self.death_timeout = death_timeout + self.death_timeout = parse_timedelta(death_timeout) self.preload = preload if self.preload is None: self.preload = dask.config.get("distributed.worker.preload") @@ -933,7 +933,8 @@ def _start(self, addr_or_port=0): if "://" in listen_host: protocol, listen_host = listen_host.split("://") - self.name = self.name or self.address + if self.name is None: + self.name = self.address preload_modules( self.preload, parameter=self, @@ -976,7 +977,15 @@ def _start(self, addr_or_port=0): raise gen.Return(self) def __await__(self): - return self._start().__await__() + if self.status is not None: + + @gen.coroutine # idempotent + def _(): + raise gen.Return(self) + + return _().__await__() + else: + return self._start().__await__() def start(self, port=0): self.loop.add_callback(self._start, port) From 6339d81e8de97b551c8cc908308c19ab89037df2 Mon Sep 17 00:00:00 2001 From: Matt Nicolls <2540582+nicolls1@users.noreply.github.com> Date: Fri, 24 May 2019 14:36:05 -0500 Subject: [PATCH 24/43] Add SchedulerPlugin TaskState example (#2622) --- docs/source/plugins.rst | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/source/plugins.rst b/docs/source/plugins.rst index edf64362c5..b5f52f8843 100644 --- a/docs/source/plugins.rst +++ b/docs/source/plugins.rst @@ -49,3 +49,27 @@ for more information on RabbitMQ and how to consume the messages. scheduler.add_plugin(plugin) Run with: ``dask-scheduler --preload `` + +Accessing Full Task State +------------------------- + +If you would like to access the full :class:`distributed.scheduler.TaskState` +stored in the scheduler you can do this by passing and storing a reference to +the scheduler as so: + +.. code-block:: python + + from distributed.diagnostics.plugin import SchedulerPlugin + + class MyPlugin(SchedulerPlugin): + def __init__(self, scheduler): + self.scheduler = scheduler + + def transition(self, key, start, finish, *args, **kwargs): + # Get full TaskState + ts = self.scheduler.tasks[key] + + @click.command() + def dask_setup(scheduler): + plugin = MyPlugin(scheduler) + scheduler.add_plugin(plugin) From a818711f97fdf501823c35246b16f24fddd4035a Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 27 May 2019 14:47:10 -0500 Subject: [PATCH 25/43] Close clusters at exit (#2730) --- distributed/deploy/spec.py | 15 +++++++++++++-- distributed/deploy/tests/test_spec_cluster.py | 11 +++++++++++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/distributed/deploy/spec.py b/distributed/deploy/spec.py index 9a4385e505..ad0aea25f6 100644 --- a/distributed/deploy/spec.py +++ b/distributed/deploy/spec.py @@ -1,4 +1,5 @@ import asyncio +import atexit import weakref from tornado import gen @@ -97,6 +98,8 @@ class does handle all of the logic around asynchronously cleanly setting up specifications into the same dictionary. """ + _instances = weakref.WeakSet() + def __init__( self, workers=None, @@ -133,6 +136,7 @@ def __init__( loop=self.loop, **self.scheduler_spec["options"] ) self.status = "created" + self._instances.add(self) self._correct_state_waiting = None if not self.asynchronous: @@ -248,9 +252,9 @@ async def _close(self): self.status = "closed" - def close(self): + def close(self, timeout=None): with ignoring(RuntimeError): # loop closed during process shutdown - return self.sync(self._close) + return self.sync(self._close, callback_timeout=timeout) def __del__(self): if self.status != "closed": @@ -295,3 +299,10 @@ def __repr__(self): self.scheduler_address, len(self.workers), ) + + +@atexit.register +def close_clusters(): + for cluster in list(SpecCluster._instances): + with ignoring(gen.TimeoutError): + cluster.close(timeout=10) diff --git a/distributed/deploy/tests/test_spec_cluster.py b/distributed/deploy/tests/test_spec_cluster.py index cfc1242727..ac5706afe1 100644 --- a/distributed/deploy/tests/test_spec_cluster.py +++ b/distributed/deploy/tests/test_spec_cluster.py @@ -1,4 +1,5 @@ from dask.distributed import SpecCluster, Worker, Client, Scheduler +from distributed.deploy.spec import close_clusters from distributed.utils_test import loop # noqa: F401 import pytest @@ -113,3 +114,13 @@ async def test_broken_worker(): pass assert "Broken" in str(info.value) + + +@pytest.mark.slow +def test_spec_close_clusters(loop): + workers = {0: {"cls": Worker}} + scheduler = {"cls": Scheduler, "options": {"port": 0}} + cluster = SpecCluster(workers=workers, scheduler=scheduler, loop=loop) + assert cluster in SpecCluster._instances + close_clusters() + assert cluster.status == "closed" From d202e6253ed8ddc7919d0d4f128d88954e9859b8 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Tue, 28 May 2019 12:25:44 -0500 Subject: [PATCH 26/43] Move bokeh module to dashboard (#2724) --- distributed/bokeh.py | 1 + distributed/bokeh/__init__.py | 37 ---------- distributed/cli/dask_mpi.py | 8 +-- distributed/cli/dask_scheduler.py | 23 +++--- distributed/cli/dask_worker.py | 18 ++--- distributed/cli/tests/test_dask_scheduler.py | 46 +++++------- distributed/cli/tests/test_dask_worker.py | 56 +++++++++------ distributed/cli/tests/test_tls_cli.py | 17 ++--- distributed/client.py | 6 +- distributed/dashboard/__init__.py | 2 + .../{bokeh => dashboard}/components.py | 0 distributed/{bokeh => dashboard}/core.py | 0 .../{bokeh => dashboard}/export_tool.coffee | 0 .../{bokeh => dashboard}/export_tool.js | 0 .../{bokeh => dashboard}/export_tool.py | 0 distributed/{bokeh => dashboard}/proxy.py | 2 +- distributed/{bokeh => dashboard}/scheduler.py | 4 +- .../{bokeh => dashboard}/scheduler_html.py | 2 +- .../{bokeh => dashboard}/static/css/base.css | 0 .../static/css/status.css | 0 .../static/css/system.css | 0 .../static/images/dask-logo.svg | 0 .../static/images/fa-bars.svg | 0 .../{bokeh => dashboard}/templates/base.html | 0 .../templates/call-stack.html | 0 .../templates/json-index.html | 0 .../{bokeh => dashboard}/templates/logs.html | 0 .../{bokeh => dashboard}/templates/main.html | 0 .../templates/simple.html | 0 .../templates/status.html | 0 .../templates/system.html | 0 .../{bokeh => dashboard}/templates/task.html | 0 .../templates/worker-table.html | 0 .../templates/worker.html | 0 .../templates/workers.html | 0 distributed/dashboard/tests/test_bokeh.py | 5 ++ .../tests/test_components.py | 4 +- .../tests/test_scheduler_bokeh.py | 30 ++++---- .../tests/test_scheduler_bokeh_html.py | 29 ++++---- .../tests/test_worker_bokeh.py | 24 ++++--- .../tests/test_worker_bokeh_html.py | 10 +-- distributed/{bokeh => dashboard}/theme.yaml | 0 distributed/{bokeh => dashboard}/utils.py | 0 distributed/{bokeh => dashboard}/worker.py | 0 .../{bokeh => dashboard}/worker_html.py | 0 distributed/deploy/cluster.py | 4 +- distributed/deploy/spec.py | 4 +- distributed/deploy/tests/test_local.py | 10 +-- .../diagnostics/tests/test_eventstream.py | 8 ++- distributed/scheduler.py | 6 +- distributed/tests/test_client.py | 11 ++- distributed/tests/test_core.py | 5 +- distributed/tests/test_scheduler.py | 14 ++-- distributed/tests/test_worker.py | 22 +++--- distributed/worker.py | 6 +- setup.py | 70 ++++++++++--------- 56 files changed, 236 insertions(+), 248 deletions(-) create mode 100644 distributed/bokeh.py delete mode 100644 distributed/bokeh/__init__.py create mode 100644 distributed/dashboard/__init__.py rename distributed/{bokeh => dashboard}/components.py (100%) rename distributed/{bokeh => dashboard}/core.py (100%) rename distributed/{bokeh => dashboard}/export_tool.coffee (100%) rename distributed/{bokeh => dashboard}/export_tool.js (100%) rename distributed/{bokeh => dashboard}/export_tool.py (100%) rename distributed/{bokeh => dashboard}/proxy.py (98%) rename distributed/{bokeh => dashboard}/scheduler.py (99%) rename distributed/{bokeh => dashboard}/scheduler_html.py (99%) rename distributed/{bokeh => dashboard}/static/css/base.css (100%) rename distributed/{bokeh => dashboard}/static/css/status.css (100%) rename distributed/{bokeh => dashboard}/static/css/system.css (100%) rename distributed/{bokeh => dashboard}/static/images/dask-logo.svg (100%) rename distributed/{bokeh => dashboard}/static/images/fa-bars.svg (100%) rename distributed/{bokeh => dashboard}/templates/base.html (100%) rename distributed/{bokeh => dashboard}/templates/call-stack.html (100%) rename distributed/{bokeh => dashboard}/templates/json-index.html (100%) rename distributed/{bokeh => dashboard}/templates/logs.html (100%) rename distributed/{bokeh => dashboard}/templates/main.html (100%) rename distributed/{bokeh => dashboard}/templates/simple.html (100%) rename distributed/{bokeh => dashboard}/templates/status.html (100%) rename distributed/{bokeh => dashboard}/templates/system.html (100%) rename distributed/{bokeh => dashboard}/templates/task.html (100%) rename distributed/{bokeh => dashboard}/templates/worker-table.html (100%) rename distributed/{bokeh => dashboard}/templates/worker.html (100%) rename distributed/{bokeh => dashboard}/templates/workers.html (100%) create mode 100644 distributed/dashboard/tests/test_bokeh.py rename distributed/{bokeh => dashboard}/tests/test_components.py (92%) rename distributed/{bokeh => dashboard}/tests/test_scheduler_bokeh.py (95%) rename distributed/{bokeh => dashboard}/tests/test_scheduler_bokeh_html.py (79%) rename distributed/{bokeh => dashboard}/tests/test_worker_bokeh.py (79%) rename distributed/{bokeh => dashboard}/tests/test_worker_bokeh_html.py (75%) rename distributed/{bokeh => dashboard}/theme.yaml (100%) rename distributed/{bokeh => dashboard}/utils.py (100%) rename distributed/{bokeh => dashboard}/worker.py (100%) rename distributed/{bokeh => dashboard}/worker_html.py (100%) diff --git a/distributed/bokeh.py b/distributed/bokeh.py new file mode 100644 index 0000000000..e27bdffa33 --- /dev/null +++ b/distributed/bokeh.py @@ -0,0 +1 @@ +raise ImportError("The distributed.bokeh module has moved to distributed.dashboard") diff --git a/distributed/bokeh/__init__.py b/distributed/bokeh/__init__.py deleted file mode 100644 index 24e082fa8e..0000000000 --- a/distributed/bokeh/__init__.py +++ /dev/null @@ -1,37 +0,0 @@ -from collections import deque - -from ..metrics import time - -n = 60 -m = 100000 - -messages = { - "workers": { - "interval": 1000, - "deque": deque(maxlen=n), - "times": deque(maxlen=n), - "index": deque(maxlen=n), - "plot-data": { - "time": deque(maxlen=n), - "cpu": deque(maxlen=n), - "memory_percent": deque(maxlen=n), - "network-send": deque(maxlen=n), - "network-recv": deque(maxlen=n), - }, - }, - "tasks": {"interval": 150, "deque": deque(maxlen=100), "times": deque(maxlen=100)}, - "progress": {}, - "processing": {"processing": {}, "memory": 0, "waiting": 0}, - "task-events": { - "interval": 200, - "deque": deque(maxlen=m), - "times": deque(maxlen=m), - "index": deque(maxlen=m), - "rectangles": { - name: deque(maxlen=m) - for name in "start duration key name color worker worker_thread y alpha".split() - }, - "workers": dict(), - "last_seen": [time()], - }, -} diff --git a/distributed/cli/dask_mpi.py b/distributed/cli/dask_mpi.py index 398596508a..c7669073f7 100644 --- a/distributed/cli/dask_mpi.py +++ b/distributed/cli/dask_mpi.py @@ -7,7 +7,7 @@ from warnings import warn from distributed import Scheduler, Nanny, Worker -from distributed.bokeh.worker import BokehWorker +from distributed.dashboard import BokehWorker from distributed.cli.utils import check_python_3 from distributed.comm.addressing import uri_from_host_port from distributed.utils import get_ip_interface @@ -82,12 +82,12 @@ def main( if rank == 0 and scheduler: try: - from distributed.bokeh.scheduler import BokehScheduler + from distributed.dashboard import BokehScheduler except ImportError: services = {} else: services = { - ("bokeh", bokeh_port): partial(BokehScheduler, prefix=bokeh_prefix) + ("dashboard", bokeh_port): partial(BokehScheduler, prefix=bokeh_prefix) } scheduler = Scheduler( scheduler_file=scheduler_file, loop=loop, services=services @@ -107,7 +107,7 @@ def main( name=rank if scheduler else None, ncores=nthreads, local_dir=local_directory, - services={("bokeh", bokeh_worker_port): BokehWorker}, + services={("dashboard", bokeh_worker_port): BokehWorker}, memory_limit=memory_limit, ) addr = uri_from_host_port(host, None, 0) diff --git a/distributed/cli/dask_scheduler.py b/distributed/cli/dask_scheduler.py index 3668be684d..1f78426f63 100755 --- a/distributed/cli/dask_scheduler.py +++ b/distributed/cli/dask_scheduler.py @@ -68,27 +68,23 @@ help="Address on which to listen for diagnostics dashboard", ) @click.option( - "--bokeh/--no-bokeh", - "_bokeh", + "--dashboard/--no-dashboard", + "dashboard", default=True, show_default=True, required=False, - help="Launch Bokeh Web UI", + help="Launch the Dashboard", ) @click.option("--show/--no-show", default=False, help="Show web UI") @click.option( - "--bokeh-whitelist", - default=None, - multiple=True, - help="IP addresses to whitelist for bokeh.", + "--dashboard-prefix", type=str, default=None, help="Prefix for the dashboard app" ) -@click.option("--bokeh-prefix", type=str, default=None, help="Prefix for the bokeh app") @click.option( "--use-xheaders", type=bool, default=False, show_default=True, - help="User xheaders in bokeh app for ssl termination in header", + help="User xheaders in dashboard app for ssl termination in header", ) @click.option("--pid-file", type=str, default="", help="File to write the process PID") @click.option( @@ -119,9 +115,8 @@ def main( port, bokeh_port, show, - _bokeh, - bokeh_whitelist, - bokeh_prefix, + dashboard, + dashboard_prefix, use_xheaders, pid_file, scheduler_file, @@ -195,8 +190,8 @@ def del_pid_file(): host=host, port=port, interface=interface, - dashboard_address=dashboard_address if _bokeh else None, - service_kwargs={"bokeh": {"prefix": bokeh_prefix}}, + dashboard_address=dashboard_address if dashboard else None, + service_kwargs={"dashboard": {"prefix": dashboard_prefix}}, ) scheduler.start() if not preload: diff --git a/distributed/cli/dask_worker.py b/distributed/cli/dask_worker.py index 439bdaf4a6..c4f83f6140 100755 --- a/distributed/cli/dask_worker.py +++ b/distributed/cli/dask_worker.py @@ -70,12 +70,12 @@ help="Address on which to listen for diagnostics dashboard", ) @click.option( - "--bokeh/--no-bokeh", - "bokeh", + "--dashboard/--no-dashboard", + "dashboard", default=True, show_default=True, required=False, - help="Launch Bokeh Web UI", + help="Launch the Dashboard", ) @click.option( "--listen-address", @@ -163,7 +163,9 @@ default=None, help="Seconds to wait for a scheduler before closing", ) -@click.option("--bokeh-prefix", type=str, default="", help="Prefix for the bokeh app") +@click.option( + "--dashboard-prefix", type=str, default="", help="Prefix for the dashboard" +) @click.option( "--preload", type=str, @@ -190,7 +192,7 @@ def main( pid_file, reconnect, resources, - bokeh, + dashboard, bokeh_port, local_directory, scheduler_file, @@ -198,7 +200,7 @@ def main( death_timeout, preload, preload_argv, - bokeh_prefix, + dashboard_prefix, tls_ca_file, tls_cert, tls_key, @@ -338,8 +340,8 @@ def del_pid_file(): interface=interface, host=host, port=port, - dashboard_address=dashboard_address if bokeh else None, - service_kwargs={"bokhe": {"prefix": bokeh_prefix}}, + dashboard_address=dashboard_address if dashboard else None, + service_kwargs={"bokhe": {"prefix": dashboard_prefix}}, name=name if nprocs == 1 or not name else name + "-" + str(i), **kwargs ) diff --git a/distributed/cli/tests/test_dask_scheduler.py b/distributed/cli/tests/test_dask_scheduler.py index 26fe607b90..754082f35e 100644 --- a/distributed/cli/tests/test_dask_scheduler.py +++ b/distributed/cli/tests/test_dask_scheduler.py @@ -26,7 +26,7 @@ def test_defaults(loop): - with popen(["dask-scheduler", "--no-bokeh"]) as proc: + with popen(["dask-scheduler", "--no-dashboard"]) as proc: @gen.coroutine def f(): @@ -43,7 +43,7 @@ def f(): def test_hostport(loop): - with popen(["dask-scheduler", "--no-bokeh", "--host", "127.0.0.1:8978"]): + with popen(["dask-scheduler", "--no-dashboard", "--host", "127.0.0.1:8978"]): @gen.coroutine def f(): @@ -57,18 +57,18 @@ def f(): c.sync(f) -def test_no_bokeh(loop): +def test_no_dashboard(loop): pytest.importorskip("bokeh") - with popen(["dask-scheduler", "--no-bokeh"]) as proc: + with popen(["dask-scheduler", "--no-dashboard"]) as proc: with Client("127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c: for i in range(3): line = proc.stderr.readline() - assert b"bokeh" not in line.lower() + assert b"dashboard" not in line.lower() with pytest.raises(Exception): requests.get("http://127.0.0.1:8787/status/") -def test_bokeh(loop): +def test_dashboard(loop): pytest.importorskip("bokeh") with popen(["dask-scheduler"]) as proc: @@ -97,7 +97,7 @@ def test_bokeh(loop): requests.get("http://127.0.0.1:8787/status/") -def test_bokeh_non_standard_ports(loop): +def test_dashboard_non_standard_ports(loop): pytest.importorskip("bokeh") with popen( @@ -122,20 +122,12 @@ def test_bokeh_non_standard_ports(loop): @pytest.mark.skipif( not sys.platform.startswith("linux"), reason="Need 127.0.0.2 to mean localhost" ) -def test_bokeh_whitelist(loop): +def test_dashboard_whitelist(loop): pytest.importorskip("bokeh") with pytest.raises(Exception): requests.get("http://localhost:8787/status/").ok - with popen( - [ - "dask-scheduler", - "--bokeh-whitelist", - "127.0.0.2:8787", - "--bokeh-whitelist", - "127.0.0.3:8787", - ] - ) as proc: + with popen(["dask-scheduler"]) as proc: with Client("127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c: pass @@ -153,9 +145,9 @@ def test_bokeh_whitelist(loop): def test_multiple_workers(loop): - with popen(["dask-scheduler", "--no-bokeh"]) as s: - with popen(["dask-worker", "localhost:8786", "--no-bokeh"]) as a: - with popen(["dask-worker", "localhost:8786", "--no-bokeh"]) as b: + with popen(["dask-scheduler", "--no-dashboard"]) as s: + with popen(["dask-worker", "localhost:8786", "--no-dashboard"]) as a: + with popen(["dask-worker", "localhost:8786", "--no-dashboard"]) as b: with Client("127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c: start = time() while len(c.ncores()) < 2: @@ -180,9 +172,9 @@ def test_interface(loop): "Available interfaces are: %s." % (if_names,) ) - with popen(["dask-scheduler", "--no-bokeh", "--interface", if_name]) as s: + with popen(["dask-scheduler", "--no-dashboard", "--interface", if_name]) as s: with popen( - ["dask-worker", "127.0.0.1:8786", "--no-bokeh", "--interface", if_name] + ["dask-worker", "127.0.0.1:8786", "--no-dashboard", "--interface", if_name] ) as a: with Client("tcp://127.0.0.1:%d" % Scheduler.default_port, loop=loop) as c: start = time() @@ -217,12 +209,12 @@ def check_pidfile(proc, pidfile): assert proc.pid == pid with tmpfile() as s: - with popen(["dask-scheduler", "--pid-file", s, "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--pid-file", s, "--no-dashboard"]) as sched: check_pidfile(sched, s) with tmpfile() as w: with popen( - ["dask-worker", "127.0.0.1:8786", "--pid-file", w, "--no-bokeh"] + ["dask-worker", "127.0.0.1:8786", "--pid-file", w, "--no-dashboard"] ) as worker: check_pidfile(worker, w) @@ -230,21 +222,21 @@ def check_pidfile(proc, pidfile): def test_scheduler_port_zero(loop): with tmpfile() as fn: with popen( - ["dask-scheduler", "--no-bokeh", "--scheduler-file", fn, "--port", "0"] + ["dask-scheduler", "--no-dashboard", "--scheduler-file", fn, "--port", "0"] ) as sched: with Client(scheduler_file=fn, loop=loop) as c: assert c.scheduler.port assert c.scheduler.port != 8786 -def test_bokeh_port_zero(loop): +def test_dashboard_port_zero(loop): pytest.importorskip("bokeh") with tmpfile() as fn: with popen(["dask-scheduler", "--dashboard-address", ":0"]) as proc: count = 0 while count < 1: line = proc.stderr.readline() - if b"bokeh" in line.lower() or b"web" in line.lower(): + if b"dashboard" in line.lower(): sleep(0.01) count += 1 assert b":0" not in line diff --git a/distributed/cli/tests/test_dask_worker.py b/distributed/cli/tests/test_dask_worker.py index aac27061b2..fa62594a75 100644 --- a/distributed/cli/tests/test_dask_worker.py +++ b/distributed/cli/tests/test_dask_worker.py @@ -17,7 +17,7 @@ def test_nanny_worker_ports(loop): - with popen(["dask-scheduler", "--port", "9359", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--port", "9359", "--no-dashboard"]) as sched: with popen( [ "dask-worker", @@ -28,7 +28,7 @@ def test_nanny_worker_ports(loop): "9684", "--nanny-port", "5273", - "--no-bokeh", + "--no-dashboard", ] ) as worker: with Client("127.0.0.1:9359", loop=loop) as c: @@ -47,9 +47,15 @@ def test_nanny_worker_ports(loop): def test_memory_limit(loop): - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: with popen( - ["dask-worker", "127.0.0.1:8786", "--memory-limit", "2e3MB", "--no-bokeh"] + [ + "dask-worker", + "127.0.0.1:8786", + "--memory-limit", + "2e3MB", + "--no-dashboard", + ] ) as worker: with Client("127.0.0.1:8786", loop=loop) as c: while not c.ncores(): @@ -61,9 +67,9 @@ def test_memory_limit(loop): def test_no_nanny(loop): - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: with popen( - ["dask-worker", "127.0.0.1:8786", "--no-nanny", "--no-bokeh"] + ["dask-worker", "127.0.0.1:8786", "--no-nanny", "--no-dashboard"] ) as worker: assert any(b"Registered" in worker.stderr.readline() for i in range(15)) @@ -71,7 +77,7 @@ def test_no_nanny(loop): @pytest.mark.slow @pytest.mark.parametrize("nanny", ["--nanny", "--no-nanny"]) def test_no_reconnect(nanny, loop): - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: wait_for_port(("127.0.0.1", 8786)) with popen( [ @@ -79,7 +85,7 @@ def test_no_reconnect(nanny, loop): "tcp://127.0.0.1:8786", "--no-reconnect", nanny, - "--no-bokeh", + "--no-dashboard", ] ) as worker: sleep(2) @@ -91,12 +97,12 @@ def test_no_reconnect(nanny, loop): def test_resources(loop): - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: with popen( [ "dask-worker", "tcp://127.0.0.1:8786", - "--no-bokeh", + "--no-dashboard", "--resources", "A=1 B=2,C=3", ] @@ -112,13 +118,13 @@ def test_resources(loop): @pytest.mark.parametrize("nanny", ["--nanny", "--no-nanny"]) def test_local_directory(loop, nanny): with tmpfile() as fn: - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: with popen( [ "dask-worker", "127.0.0.1:8786", nanny, - "--no-bokeh", + "--no-dashboard", "--local-directory", fn, ] @@ -136,8 +142,12 @@ def test_local_directory(loop, nanny): @pytest.mark.parametrize("nanny", ["--nanny", "--no-nanny"]) def test_scheduler_file(loop, nanny): with tmpfile() as fn: - with popen(["dask-scheduler", "--no-bokeh", "--scheduler-file", fn]) as sched: - with popen(["dask-worker", "--scheduler-file", fn, nanny, "--no-bokeh"]): + with popen( + ["dask-scheduler", "--no-dashboard", "--scheduler-file", fn] + ) as sched: + with popen( + ["dask-worker", "--scheduler-file", fn, nanny, "--no-dashboard"] + ): with Client(scheduler_file=fn, loop=loop) as c: start = time() while not c.scheduler_info()["workers"]: @@ -147,8 +157,8 @@ def test_scheduler_file(loop, nanny): def test_scheduler_address_env(loop, monkeypatch): monkeypatch.setenv("DASK_SCHEDULER_ADDRESS", "tcp://127.0.0.1:8786") - with popen(["dask-scheduler", "--no-bokeh"]) as sched: - with popen(["dask-worker", "--no-bokeh"]): + with popen(["dask-scheduler", "--no-dashboard"]) as sched: + with popen(["dask-worker", "--no-dashboard"]): with Client(os.environ["DASK_SCHEDULER_ADDRESS"], loop=loop) as c: start = time() while not c.scheduler_info()["workers"]: @@ -157,7 +167,7 @@ def test_scheduler_address_env(loop, monkeypatch): def test_nprocs_requires_nanny(loop): - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: with popen( ["dask-worker", "127.0.0.1:8786", "--nprocs=2", "--no-nanny"] ) as worker: @@ -168,7 +178,7 @@ def test_nprocs_requires_nanny(loop): def test_nprocs_expands_name(loop): - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: with popen( ["dask-worker", "127.0.0.1:8786", "--nprocs", "2", "--name", "foo"] ) as worker: @@ -194,13 +204,13 @@ def test_nprocs_expands_name(loop): "listen_address", ["tcp://0.0.0.0:39837", "tcp://127.0.0.2:39837"] ) def test_contact_listen_address(loop, nanny, listen_address): - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: with popen( [ "dask-worker", "127.0.0.1:8786", nanny, - "--no-bokeh", + "--no-dashboard", "--contact-address", "tcp://127.0.0.2:39837", "--listen-address", @@ -228,9 +238,9 @@ def func(dask_worker): @pytest.mark.parametrize("nanny", ["--nanny", "--no-nanny"]) @pytest.mark.parametrize("host", ["127.0.0.2", "0.0.0.0"]) def test_respect_host_listen_address(loop, nanny, host): - with popen(["dask-scheduler", "--no-bokeh"]) as sched: + with popen(["dask-scheduler", "--no-dashboard"]) as sched: with popen( - ["dask-worker", "127.0.0.1:8786", nanny, "--no-bokeh", "--host", host] + ["dask-worker", "127.0.0.1:8786", nanny, "--no-dashboard", "--host", host] ) as worker: with Client("127.0.0.1:8786") as client: while not client.ncores(): @@ -247,7 +257,7 @@ def func(dask_worker): assert all(host in v for v in listen_addresses.values()) -def test_bokeh_non_standard_ports(loop): +def test_dashboard_non_standard_ports(loop): pytest.importorskip("bokeh") try: import jupyter_server_proxy # noqa: F401 diff --git a/distributed/cli/tests/test_tls_cli.py b/distributed/cli/tests/test_tls_cli.py index d983039c96..4663a9b38f 100644 --- a/distributed/cli/tests/test_tls_cli.py +++ b/distributed/cli/tests/test_tls_cli.py @@ -33,9 +33,9 @@ def wait_for_cores(c, ncores=1): def test_basic(loop): - with popen(["dask-scheduler", "--no-bokeh"] + tls_args) as s: + with popen(["dask-scheduler", "--no-dashboard"] + tls_args) as s: with popen( - ["dask-worker", "--no-bokeh", "tls://127.0.0.1:8786"] + tls_args + ["dask-worker", "--no-dashboard", "tls://127.0.0.1:8786"] + tls_args ) as w: with Client( "tls://127.0.0.1:8786", loop=loop, security=tls_security() @@ -44,9 +44,10 @@ def test_basic(loop): def test_nanny(loop): - with popen(["dask-scheduler", "--no-bokeh"] + tls_args) as s: + with popen(["dask-scheduler", "--no-dashboard"] + tls_args) as s: with popen( - ["dask-worker", "--no-bokeh", "--nanny", "tls://127.0.0.1:8786"] + tls_args + ["dask-worker", "--no-dashboard", "--nanny", "tls://127.0.0.1:8786"] + + tls_args ) as w: with Client( "tls://127.0.0.1:8786", loop=loop, security=tls_security() @@ -55,9 +56,9 @@ def test_nanny(loop): def test_separate_key_cert(loop): - with popen(["dask-scheduler", "--no-bokeh"] + tls_args_2) as s: + with popen(["dask-scheduler", "--no-dashboard"] + tls_args_2) as s: with popen( - ["dask-worker", "--no-bokeh", "tls://127.0.0.1:8786"] + tls_args_2 + ["dask-worker", "--no-dashboard", "tls://127.0.0.1:8786"] + tls_args_2 ) as w: with Client( "tls://127.0.0.1:8786", loop=loop, security=tls_security() @@ -67,8 +68,8 @@ def test_separate_key_cert(loop): def test_use_config_file(loop): with new_config_file(tls_only_config()): - with popen(["dask-scheduler", "--no-bokeh", "--host", "tls://"]) as s: - with popen(["dask-worker", "--no-bokeh", "tls://127.0.0.1:8786"]) as w: + with popen(["dask-scheduler", "--no-dashboard", "--host", "tls://"]) as s: + with popen(["dask-worker", "--no-dashboard", "tls://127.0.0.1:8786"]) as w: with Client( "tls://127.0.0.1:8786", loop=loop, security=tls_security() ) as c: diff --git a/distributed/client.py b/distributed/client.py index afe6f6ef39..d924b608c6 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -812,9 +812,9 @@ def _repr_html_(self): text = ( "

Client

\n" "
    \n" "
  • Scheduler: not connected\n" ) - if info and "bokeh" in info["services"]: + if info and "dashboard" in info["services"]: protocol, rest = scheduler.address.split("://") - port = info["services"]["bokeh"] + port = info["services"]["dashboard"] if protocol == "inproc": host = "localhost" else: @@ -3852,7 +3852,7 @@ def _get_task_stream( from .diagnostics.task_stream import rectangles rects = rectangles(msgs) - from .bokeh.components import task_stream_figure + from .dashboard.components import task_stream_figure source, figure = task_stream_figure(sizing_mode="stretch_both") source.data.update(rects) diff --git a/distributed/dashboard/__init__.py b/distributed/dashboard/__init__.py new file mode 100644 index 0000000000..675963b146 --- /dev/null +++ b/distributed/dashboard/__init__.py @@ -0,0 +1,2 @@ +from .scheduler import BokehScheduler +from .worker import BokehWorker diff --git a/distributed/bokeh/components.py b/distributed/dashboard/components.py similarity index 100% rename from distributed/bokeh/components.py rename to distributed/dashboard/components.py diff --git a/distributed/bokeh/core.py b/distributed/dashboard/core.py similarity index 100% rename from distributed/bokeh/core.py rename to distributed/dashboard/core.py diff --git a/distributed/bokeh/export_tool.coffee b/distributed/dashboard/export_tool.coffee similarity index 100% rename from distributed/bokeh/export_tool.coffee rename to distributed/dashboard/export_tool.coffee diff --git a/distributed/bokeh/export_tool.js b/distributed/dashboard/export_tool.js similarity index 100% rename from distributed/bokeh/export_tool.js rename to distributed/dashboard/export_tool.js diff --git a/distributed/bokeh/export_tool.py b/distributed/dashboard/export_tool.py similarity index 100% rename from distributed/bokeh/export_tool.py rename to distributed/dashboard/export_tool.py diff --git a/distributed/bokeh/proxy.py b/distributed/dashboard/proxy.py similarity index 98% rename from distributed/bokeh/proxy.py rename to distributed/dashboard/proxy.py index 9353e38311..89f9f87aae 100644 --- a/distributed/bokeh/proxy.py +++ b/distributed/dashboard/proxy.py @@ -124,7 +124,7 @@ def check_worker_dashboard_exits(scheduler, worker): addr, port = worker.split(":") workers = list(scheduler.workers.values()) for w in workers: - bokeh_port = w.services.get("bokeh", "") + bokeh_port = w.services.get("dashboard", "") if addr == w.host and port == str(bokeh_port): return True return False diff --git a/distributed/bokeh/scheduler.py b/distributed/dashboard/scheduler.py similarity index 99% rename from distributed/bokeh/scheduler.py rename to distributed/dashboard/scheduler.py index e0f5bfffab..6476d3aa6e 100644 --- a/distributed/bokeh/scheduler.py +++ b/distributed/dashboard/scheduler.py @@ -170,7 +170,7 @@ def update(self): workers = list(self.scheduler.workers.values()) dashboard_host = [ws.host for ws in workers] - dashboard_port = [ws.services.get("bokeh", "") for ws in workers] + dashboard_port = [ws.services.get("dashboard", "") for ws in workers] y = list(range(len(workers))) occupancy = [ws.occupancy for ws in workers] @@ -403,7 +403,7 @@ def update(self): workers = list(self.scheduler.workers.values()) dashboard_host = [ws.host for ws in workers] - dashboard_port = [ws.services.get("bokeh", "") for ws in workers] + dashboard_port = [ws.services.get("dashboard", "") for ws in workers] y = list(range(len(workers))) nprocessing = [len(ws.processing) for ws in workers] diff --git a/distributed/bokeh/scheduler_html.py b/distributed/dashboard/scheduler_html.py similarity index 99% rename from distributed/bokeh/scheduler_html.py rename to distributed/dashboard/scheduler_html.py index 1d3635c37c..5f481f783b 100644 --- a/distributed/bokeh/scheduler_html.py +++ b/distributed/dashboard/scheduler_html.py @@ -177,7 +177,7 @@ def get(self): class IndividualPlots(RequestHandler): def get(self): - bokeh_server = self.server.services["bokeh"] + bokeh_server = self.server.services["dashboard"] result = { uri.strip("/").replace("-", " ").title(): uri for uri in bokeh_server.apps diff --git a/distributed/bokeh/static/css/base.css b/distributed/dashboard/static/css/base.css similarity index 100% rename from distributed/bokeh/static/css/base.css rename to distributed/dashboard/static/css/base.css diff --git a/distributed/bokeh/static/css/status.css b/distributed/dashboard/static/css/status.css similarity index 100% rename from distributed/bokeh/static/css/status.css rename to distributed/dashboard/static/css/status.css diff --git a/distributed/bokeh/static/css/system.css b/distributed/dashboard/static/css/system.css similarity index 100% rename from distributed/bokeh/static/css/system.css rename to distributed/dashboard/static/css/system.css diff --git a/distributed/bokeh/static/images/dask-logo.svg b/distributed/dashboard/static/images/dask-logo.svg similarity index 100% rename from distributed/bokeh/static/images/dask-logo.svg rename to distributed/dashboard/static/images/dask-logo.svg diff --git a/distributed/bokeh/static/images/fa-bars.svg b/distributed/dashboard/static/images/fa-bars.svg similarity index 100% rename from distributed/bokeh/static/images/fa-bars.svg rename to distributed/dashboard/static/images/fa-bars.svg diff --git a/distributed/bokeh/templates/base.html b/distributed/dashboard/templates/base.html similarity index 100% rename from distributed/bokeh/templates/base.html rename to distributed/dashboard/templates/base.html diff --git a/distributed/bokeh/templates/call-stack.html b/distributed/dashboard/templates/call-stack.html similarity index 100% rename from distributed/bokeh/templates/call-stack.html rename to distributed/dashboard/templates/call-stack.html diff --git a/distributed/bokeh/templates/json-index.html b/distributed/dashboard/templates/json-index.html similarity index 100% rename from distributed/bokeh/templates/json-index.html rename to distributed/dashboard/templates/json-index.html diff --git a/distributed/bokeh/templates/logs.html b/distributed/dashboard/templates/logs.html similarity index 100% rename from distributed/bokeh/templates/logs.html rename to distributed/dashboard/templates/logs.html diff --git a/distributed/bokeh/templates/main.html b/distributed/dashboard/templates/main.html similarity index 100% rename from distributed/bokeh/templates/main.html rename to distributed/dashboard/templates/main.html diff --git a/distributed/bokeh/templates/simple.html b/distributed/dashboard/templates/simple.html similarity index 100% rename from distributed/bokeh/templates/simple.html rename to distributed/dashboard/templates/simple.html diff --git a/distributed/bokeh/templates/status.html b/distributed/dashboard/templates/status.html similarity index 100% rename from distributed/bokeh/templates/status.html rename to distributed/dashboard/templates/status.html diff --git a/distributed/bokeh/templates/system.html b/distributed/dashboard/templates/system.html similarity index 100% rename from distributed/bokeh/templates/system.html rename to distributed/dashboard/templates/system.html diff --git a/distributed/bokeh/templates/task.html b/distributed/dashboard/templates/task.html similarity index 100% rename from distributed/bokeh/templates/task.html rename to distributed/dashboard/templates/task.html diff --git a/distributed/bokeh/templates/worker-table.html b/distributed/dashboard/templates/worker-table.html similarity index 100% rename from distributed/bokeh/templates/worker-table.html rename to distributed/dashboard/templates/worker-table.html diff --git a/distributed/bokeh/templates/worker.html b/distributed/dashboard/templates/worker.html similarity index 100% rename from distributed/bokeh/templates/worker.html rename to distributed/dashboard/templates/worker.html diff --git a/distributed/bokeh/templates/workers.html b/distributed/dashboard/templates/workers.html similarity index 100% rename from distributed/bokeh/templates/workers.html rename to distributed/dashboard/templates/workers.html diff --git a/distributed/dashboard/tests/test_bokeh.py b/distributed/dashboard/tests/test_bokeh.py new file mode 100644 index 0000000000..363272be5f --- /dev/null +++ b/distributed/dashboard/tests/test_bokeh.py @@ -0,0 +1,5 @@ +def test_old_import(): + try: + from distributed.bokeh import BokehScheduler # noqa: F401 + except ImportError as e: + assert "distributed.dashboard" in str(e) diff --git a/distributed/bokeh/tests/test_components.py b/distributed/dashboard/tests/test_components.py similarity index 92% rename from distributed/bokeh/tests/test_components.py rename to distributed/dashboard/tests/test_components.py index 028f209b41..d441db57ae 100644 --- a/distributed/bokeh/tests/test_components.py +++ b/distributed/dashboard/tests/test_components.py @@ -7,10 +7,9 @@ from bokeh.models import ColumnDataSource, Model from tornado import gen -from distributed.bokeh import messages from distributed.utils_test import slowinc, gen_cluster -from distributed.bokeh.components import ( +from distributed.dashboard.components import ( TaskStream, MemoryUsage, Processing, @@ -24,7 +23,6 @@ def test_basic(Component): c = Component() assert isinstance(c.source, ColumnDataSource) assert isinstance(c.root, Model) - c.update(messages) @gen_cluster(client=True, check_new_threads=False) diff --git a/distributed/bokeh/tests/test_scheduler_bokeh.py b/distributed/dashboard/tests/test_scheduler_bokeh.py similarity index 95% rename from distributed/bokeh/tests/test_scheduler_bokeh.py rename to distributed/dashboard/tests/test_scheduler_bokeh.py index 057aa67965..f8a813514b 100644 --- a/distributed/bokeh/tests/test_scheduler_bokeh.py +++ b/distributed/dashboard/tests/test_scheduler_bokeh.py @@ -17,8 +17,8 @@ from distributed.client import wait from distributed.metrics import time from distributed.utils_test import gen_cluster, inc, dec, slowinc, div -from distributed.bokeh.worker import Counters, BokehWorker -from distributed.bokeh.scheduler import ( +from distributed.dashboard.worker import Counters, BokehWorker +from distributed.dashboard.scheduler import ( BokehScheduler, SystemMonitor, Occupancy, @@ -36,7 +36,7 @@ ProfileServer, ) -from distributed.bokeh import scheduler +from distributed.dashboard import scheduler scheduler.PROFILING = False @@ -44,10 +44,12 @@ @pytest.mark.skipif( sys.version_info[0] == 2, reason="https://github.com/bokeh/bokeh/issues/5494" ) -@gen_cluster(client=True, scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}) +@gen_cluster( + client=True, scheduler_kwargs={"services": {("dashboard", 0): BokehScheduler}} +) def test_simple(c, s, a, b): - assert isinstance(s.services["bokeh"], BokehScheduler) - port = s.services["bokeh"].port + assert isinstance(s.services["dashboard"], BokehScheduler) + port = s.services["dashboard"].port future = c.submit(sleep, 1) yield gen.sleep(0.1) @@ -80,7 +82,7 @@ def test_simple(c, s, a, b): assert response -@gen_cluster(client=True, worker_kwargs=dict(services={"bokeh": BokehWorker})) +@gen_cluster(client=True, worker_kwargs=dict(services={"dashboard": BokehWorker})) def test_basic(c, s, a, b): for component in [SystemMonitor, Occupancy, StealingTimeSeries]: ss = component(s) @@ -573,11 +575,13 @@ def test_profile_server(c, s, a, b): assert time() < start + 2 -@gen_cluster(client=True, scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}) +@gen_cluster( + client=True, scheduler_kwargs={"services": {("dashboard", 0): BokehScheduler}} +) def test_root_redirect(c, s, a, b): http_client = AsyncHTTPClient() response = yield http_client.fetch( - "http://localhost:%d/" % s.services["bokeh"].port + "http://localhost:%d/" % s.services["dashboard"].port ) assert response.code == 200 assert "/status" in response.effective_url @@ -585,8 +589,8 @@ def test_root_redirect(c, s, a, b): @gen_cluster( client=True, - scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}, - worker_kwargs={"services": {"bokeh": BokehWorker}}, + scheduler_kwargs={"services": {("dashboard", 0): BokehScheduler}}, + worker_kwargs={"services": {"dashboard": BokehWorker}}, timeout=180, ) def test_proxy_to_workers(c, s, a, b): @@ -597,7 +601,7 @@ def test_proxy_to_workers(c, s, a, b): except ImportError: proxy_exists = False - dashboard_port = s.services["bokeh"].port + dashboard_port = s.services["dashboard"].port http_client = AsyncHTTPClient() response = yield http_client.fetch("http://localhost:%d/" % dashboard_port) assert response.code == 200 @@ -605,7 +609,7 @@ def test_proxy_to_workers(c, s, a, b): for w in [a, b]: host = w.ip - port = w.service_ports["bokeh"] + port = w.service_ports["dashboard"] proxy_url = "http://localhost:%d/proxy/%s/%s/status" % ( dashboard_port, port, diff --git a/distributed/bokeh/tests/test_scheduler_bokeh_html.py b/distributed/dashboard/tests/test_scheduler_bokeh_html.py similarity index 79% rename from distributed/bokeh/tests/test_scheduler_bokeh_html.py rename to distributed/dashboard/tests/test_scheduler_bokeh_html.py index 691121f751..f872d02dc8 100644 --- a/distributed/bokeh/tests/test_scheduler_bokeh_html.py +++ b/distributed/dashboard/tests/test_scheduler_bokeh_html.py @@ -13,14 +13,13 @@ from dask.sizeof import sizeof from distributed.utils_test import gen_cluster, slowinc, inc -from distributed.bokeh.scheduler import BokehScheduler -from distributed.bokeh.worker import BokehWorker +from distributed.dashboard import BokehScheduler, BokehWorker @gen_cluster( client=True, - scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}, - worker_kwargs={"services": {"bokeh": BokehWorker}}, + scheduler_kwargs={"services": {("dashboard", 0): BokehScheduler}}, + worker_kwargs={"services": {"dashboard": BokehWorker}}, ) def test_connect(c, s, a, b): future = c.submit(lambda x: x + 1, 1) @@ -41,7 +40,7 @@ def test_connect(c, s, a, b): "individual-plots.json", ]: response = yield http_client.fetch( - "http://localhost:%d/%s" % (s.services["bokeh"].port, suffix) + "http://localhost:%d/%s" % (s.services["dashboard"].port, suffix) ) assert response.code == 200 body = response.body.decode() @@ -54,13 +53,15 @@ def test_connect(c, s, a, b): @gen_cluster( client=True, - scheduler_kwargs={"services": {("bokeh", 0): (BokehScheduler, {"prefix": "/foo"})}}, + scheduler_kwargs={ + "services": {("dashboard", 0): (BokehScheduler, {"prefix": "/foo"})} + }, ) def test_prefix(c, s, a, b): http_client = AsyncHTTPClient() for suffix in ["foo/info/main/workers.html", "foo/json/index.html", "foo/system"]: response = yield http_client.fetch( - "http://localhost:%d/%s" % (s.services["bokeh"].port, suffix) + "http://localhost:%d/%s" % (s.services["dashboard"].port, suffix) ) assert response.code == 200 body = response.body.decode() @@ -73,7 +74,7 @@ def test_prefix(c, s, a, b): @gen_cluster( client=True, check_new_threads=False, - scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}, + scheduler_kwargs={"services": {("dashboard", 0): BokehScheduler}}, ) def test_prometheus(c, s, a, b): pytest.importorskip("prometheus_client") @@ -85,7 +86,7 @@ def test_prometheus(c, s, a, b): # prometheus_client errors for _ in range(2): response = yield http_client.fetch( - "http://localhost:%d/metrics" % s.services["bokeh"].port + "http://localhost:%d/metrics" % s.services["dashboard"].port ) assert response.code == 200 assert response.headers["Content-Type"] == "text/plain; version=0.0.4" @@ -98,13 +99,13 @@ def test_prometheus(c, s, a, b): @gen_cluster( client=True, check_new_threads=False, - scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}, + scheduler_kwargs={"services": {("dashboard", 0): BokehScheduler}}, ) def test_health(c, s, a, b): http_client = AsyncHTTPClient() response = yield http_client.fetch( - "http://localhost:%d/health" % s.services["bokeh"].port + "http://localhost:%d/health" % s.services["dashboard"].port ) assert response.code == 200 assert response.headers["Content-Type"] == "text/plain" @@ -113,7 +114,9 @@ def test_health(c, s, a, b): assert txt == "ok" -@gen_cluster(client=True, scheduler_kwargs={"services": {("bokeh", 0): BokehScheduler}}) +@gen_cluster( + client=True, scheduler_kwargs={"services": {("dashboard", 0): BokehScheduler}} +) def test_task_page(c, s, a, b): future = c.submit(lambda x: x + 1, 1, workers=a.address) x = c.submit(inc, 1) @@ -122,7 +125,7 @@ def test_task_page(c, s, a, b): "info/task/" + url_escape(future.key) + ".html", response = yield http_client.fetch( - "http://localhost:%d/info/task/" % s.services["bokeh"].port + "http://localhost:%d/info/task/" % s.services["dashboard"].port + url_escape(future.key) + ".html" ) diff --git a/distributed/bokeh/tests/test_worker_bokeh.py b/distributed/dashboard/tests/test_worker_bokeh.py similarity index 79% rename from distributed/bokeh/tests/test_worker_bokeh.py rename to distributed/dashboard/tests/test_worker_bokeh.py index 03a7ed3861..11699d9ac8 100644 --- a/distributed/bokeh/tests/test_worker_bokeh.py +++ b/distributed/dashboard/tests/test_worker_bokeh.py @@ -14,7 +14,7 @@ from distributed.client import wait from distributed.metrics import time from distributed.utils_test import gen_cluster, inc, dec -from distributed.bokeh.worker import ( +from distributed.dashboard.worker import ( BokehWorker, StateTable, CrossFilter, @@ -29,10 +29,10 @@ @pytest.mark.skipif( sys.version_info[0] == 2, reason="https://github.com/bokeh/bokeh/issues/5494" ) -@gen_cluster(client=True, worker_kwargs={"services": {("bokeh", 0): BokehWorker}}) +@gen_cluster(client=True, worker_kwargs={"services": {("dashboard", 0): BokehWorker}}) def test_simple(c, s, a, b): - assert s.workers[a.address].services == {"bokeh": a.services["bokeh"].port} - assert s.workers[b.address].services == {"bokeh": b.services["bokeh"].port} + assert s.workers[a.address].services == {"dashboard": a.services["dashboard"].port} + assert s.workers[b.address].services == {"dashboard": b.services["dashboard"].port} future = c.submit(sleep, 1) yield gen.sleep(0.1) @@ -40,15 +40,17 @@ def test_simple(c, s, a, b): http_client = AsyncHTTPClient() for suffix in ["main", "crossfilter", "system"]: response = yield http_client.fetch( - "http://localhost:%d/%s" % (a.services["bokeh"].port, suffix) + "http://localhost:%d/%s" % (a.services["dashboard"].port, suffix) ) assert "bokeh" in response.body.decode().lower() -@gen_cluster(client=True, worker_kwargs={"services": {("bokeh", 0): (BokehWorker, {})}}) +@gen_cluster( + client=True, worker_kwargs={"services": {("dashboard", 0): (BokehWorker, {})}} +) def test_services_kwargs(c, s, a, b): - assert s.workers[a.address].services == {"bokeh": a.services["bokeh"].port} - assert isinstance(a.services["bokeh"], BokehWorker) + assert s.workers[a.address].services == {"dashboard": a.services["dashboard"].port} + assert isinstance(a.services["dashboard"], BokehWorker) @gen_cluster(client=True) @@ -139,15 +141,15 @@ def test_CommunicatingStream(c, s, a, b): @gen_cluster( client=True, check_new_threads=False, - worker_kwargs={"services": {("bokeh", 0): BokehWorker}}, + worker_kwargs={"services": {("dashboard", 0): BokehWorker}}, ) def test_prometheus(c, s, a, b): pytest.importorskip("prometheus_client") - assert s.workers[a.address].services == {"bokeh": a.services["bokeh"].port} + assert s.workers[a.address].services == {"dashboard": a.services["dashboard"].port} http_client = AsyncHTTPClient() for suffix in ["metrics"]: response = yield http_client.fetch( - "http://localhost:%d/%s" % (a.services["bokeh"].port, suffix) + "http://localhost:%d/%s" % (a.services["dashboard"].port, suffix) ) assert response.code == 200 diff --git a/distributed/bokeh/tests/test_worker_bokeh_html.py b/distributed/dashboard/tests/test_worker_bokeh_html.py similarity index 75% rename from distributed/bokeh/tests/test_worker_bokeh_html.py rename to distributed/dashboard/tests/test_worker_bokeh_html.py index d59fec8d2d..99916b3fdc 100644 --- a/distributed/bokeh/tests/test_worker_bokeh_html.py +++ b/distributed/dashboard/tests/test_worker_bokeh_html.py @@ -4,10 +4,10 @@ from tornado.httpclient import AsyncHTTPClient from distributed.utils_test import gen_cluster -from distributed.bokeh.worker import BokehWorker +from distributed.dashboard import BokehWorker -@gen_cluster(client=True, worker_kwargs={"services": {("bokeh", 0): BokehWorker}}) +@gen_cluster(client=True, worker_kwargs={"services": {("dashboard", 0): BokehWorker}}) def test_prometheus(c, s, a, b): pytest.importorskip("prometheus_client") from prometheus_client.parser import text_string_to_metric_families @@ -18,7 +18,7 @@ def test_prometheus(c, s, a, b): # prometheus_client errors for _ in range(2): response = yield http_client.fetch( - "http://localhost:%d/metrics" % a.services["bokeh"].port + "http://localhost:%d/metrics" % a.services["dashboard"].port ) assert response.code == 200 assert response.headers["Content-Type"] == "text/plain; version=0.0.4" @@ -28,12 +28,12 @@ def test_prometheus(c, s, a, b): assert len(families) > 0 -@gen_cluster(client=True, worker_kwargs={"services": {("bokeh", 0): BokehWorker}}) +@gen_cluster(client=True, worker_kwargs={"services": {("dashboard", 0): BokehWorker}}) def test_health(c, s, a, b): http_client = AsyncHTTPClient() response = yield http_client.fetch( - "http://localhost:%d/health" % a.services["bokeh"].port + "http://localhost:%d/health" % a.services["dashboard"].port ) assert response.code == 200 assert response.headers["Content-Type"] == "text/plain" diff --git a/distributed/bokeh/theme.yaml b/distributed/dashboard/theme.yaml similarity index 100% rename from distributed/bokeh/theme.yaml rename to distributed/dashboard/theme.yaml diff --git a/distributed/bokeh/utils.py b/distributed/dashboard/utils.py similarity index 100% rename from distributed/bokeh/utils.py rename to distributed/dashboard/utils.py diff --git a/distributed/bokeh/worker.py b/distributed/dashboard/worker.py similarity index 100% rename from distributed/bokeh/worker.py rename to distributed/dashboard/worker.py diff --git a/distributed/bokeh/worker_html.py b/distributed/dashboard/worker_html.py similarity index 100% rename from distributed/bokeh/worker_html.py rename to distributed/dashboard/worker_html.py diff --git a/distributed/deploy/cluster.py b/distributed/deploy/cluster.py index 8425b836a4..69cc5be9fa 100644 --- a/distributed/deploy/cluster.py +++ b/distributed/deploy/cluster.py @@ -88,7 +88,7 @@ def scheduler_address(self): def dashboard_link(self): template = dask.config.get("distributed.dashboard.link") host = self.scheduler.address.split("://")[1].split(":")[0] - port = self.scheduler.services["bokeh"].port + port = self.scheduler.services["dashboard"].port return template.format(host=host, port=port, **os.environ) def scale(self, n): @@ -165,7 +165,7 @@ def _widget(self): layout = Layout(width="150px") - if "bokeh" in self.scheduler.services: + if "dashboard" in self.scheduler.services: link = self.dashboard_link link = '

    Dashboard: %s

    \n' % ( link, diff --git a/distributed/deploy/spec.py b/distributed/deploy/spec.py index ad0aea25f6..d5a954effc 100644 --- a/distributed/deploy/spec.py +++ b/distributed/deploy/spec.py @@ -112,11 +112,11 @@ def __init__( self._created = weakref.WeakSet() if scheduler is None: try: - from distributed.bokeh.scheduler import BokehScheduler + from distributed.dashboard import BokehScheduler except ImportError: services = {} else: - services = {("bokeh", 8787): BokehScheduler} + services = {("dashboard", 8787): BokehScheduler} scheduler = {"cls": Scheduler, "options": {"services": services}} self.scheduler_spec = scheduler diff --git a/distributed/deploy/tests/test_local.py b/distributed/deploy/tests/test_local.py index 4498611d7e..6f9a4a0324 100644 --- a/distributed/deploy/tests/test_local.py +++ b/distributed/deploy/tests/test_local.py @@ -198,8 +198,8 @@ def test_duplicate_clients(): with pytest.warns(Exception) as info: c2 = yield Client(processes=False, silence_logs=False, dashboard_address=9876) - assert "bokeh" in c1.cluster.scheduler.services - assert "bokeh" in c2.cluster.scheduler.services + assert "dashboard" in c1.cluster.scheduler.services + assert "dashboard" in c2.cluster.scheduler.services assert any( all( @@ -341,7 +341,7 @@ def test_bokeh(loop, processes): processes=processes, dashboard_address=0, ) as c: - bokeh_port = c.scheduler.services["bokeh"].port + bokeh_port = c.scheduler.services["dashboard"].port url = "http://127.0.0.1:%d/status/" % bokeh_port start = time() while True: @@ -485,10 +485,10 @@ def test_bokeh_kwargs(loop): silence_logs=False, loop=loop, dashboard_address=0, - service_kwargs={"bokeh": {"prefix": "/foo"}}, + service_kwargs={"dashboard": {"prefix": "/foo"}}, ) as c: - bs = c.scheduler.services["bokeh"] + bs = c.scheduler.services["dashboard"] assert bs.prefix == "/foo" diff --git a/distributed/diagnostics/tests/test_eventstream.py b/distributed/diagnostics/tests/test_eventstream.py index 0995d80db2..7ec646d7e9 100644 --- a/distributed/diagnostics/tests/test_eventstream.py +++ b/distributed/diagnostics/tests/test_eventstream.py @@ -1,6 +1,6 @@ from __future__ import print_function, division, absolute_import -from copy import deepcopy +import collections import pytest from tornado import gen @@ -26,10 +26,12 @@ def test_eventstream(c, s, *workers): assert len(es.buffer) == 11 - from distributed.bokeh import messages from distributed.diagnostics.progress_stream import task_stream_append - lists = deepcopy(messages["task-events"]["rectangles"]) + lists = { + name: collections.deque(maxlen=100) + for name in "start duration key name color worker worker_thread y alpha".split() + } workers = dict() for msg in es.buffer: task_stream_append(lists, msg, workers) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 991ff1a210..9db6477aeb 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -878,13 +878,13 @@ def __init__( if dashboard_address is not None: try: - from distributed.bokeh.scheduler import BokehScheduler + from distributed.dashboard import BokehScheduler except ImportError: logger.debug("To start diagnostics web server please install Bokeh") else: - self.service_specs[("bokeh", dashboard_address)] = ( + self.service_specs[("dashboard", dashboard_address)] = ( BokehScheduler, - (service_kwargs or {}).get("bokeh", {}), + (service_kwargs or {}).get("dashboard", {}), ) # Communication state diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 28c2f939eb..c731ae6e5a 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -3506,7 +3506,7 @@ def test_reconnect(loop): "127.0.0.1", "--port", "9393", - "--no-bokeh", + "--no-dashboard", ] with popen(scheduler_cli) as s: c = Client("127.0.0.1:9393", loop=loop) @@ -5221,12 +5221,11 @@ def test_quiet_scheduler_loss(c, s): @pytest.mark.skipif("USER" not in os.environ, reason="no USER env variable") def test_diagnostics_link_env_variable(loop): pytest.importorskip("bokeh") - from distributed.bokeh.scheduler import BokehScheduler + from distributed.dashboard import BokehScheduler - with cluster(scheduler_kwargs={"services": {("bokeh", 12355): BokehScheduler}}) as ( - s, - [a, b], - ): + with cluster( + scheduler_kwargs={"services": {("dashboard", 12355): BokehScheduler}} + ) as (s, [a, b]): with Client(s["address"], loop=loop) as c: with dask.config.set( {"distributed.dashboard.link": "http://foo-{USER}:{port}/status"} diff --git a/distributed/tests/test_core.py b/distributed/tests/test_core.py index 4b3c0ac0ad..f53340d100 100644 --- a/distributed/tests/test_core.py +++ b/distributed/tests/test_core.py @@ -324,7 +324,10 @@ def check_rpc_message_lifetime(*listen_args): obj = CountedObject() assert CountedObject.n_instances == 1 del obj - assert CountedObject.n_instances == 0 + start = time() + while CountedObject.n_instances != 0: + yield gen.sleep(0.01) + assert time() < start + 1 with rpc(server.address) as remote: obj = CountedObject() diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index f5ce276b8d..6df271ae34 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -1174,7 +1174,7 @@ def test_correct_bad_time_estimate(c, s, *workers): @gen_test() def test_service_hosts(): pytest.importorskip("bokeh") - from distributed.bokeh.scheduler import BokehScheduler + from distributed.dashboard import BokehScheduler port = 0 for url, expected in [ @@ -1182,12 +1182,12 @@ def test_service_hosts(): ("tcp://127.0.0.1", "127.0.0.1"), ("tcp://127.0.0.1:38275", "127.0.0.1"), ]: - services = {("bokeh", port): BokehScheduler} + services = {("dashboard", port): BokehScheduler} s = Scheduler(services=services) yield s.start(url) - sock = first(s.services["bokeh"].server._http._sockets.values()) + sock = first(s.services["dashboard"].server._http._sockets.values()) if isinstance(expected, tuple): assert sock.getsockname()[0] in expected else: @@ -1196,12 +1196,12 @@ def test_service_hosts(): port = ("127.0.0.1", 0) for url in ["tcp://0.0.0.0", "tcp://127.0.0.1", "tcp://127.0.0.1:38275"]: - services = {("bokeh", port): BokehScheduler} + services = {("dashboard", port): BokehScheduler} s = Scheduler(services=services) yield s.start(url) - sock = first(s.services["bokeh"].server._http._sockets.values()) + sock = first(s.services["dashboard"].server._http._sockets.values()) assert sock.getsockname()[0] == "127.0.0.1" yield s.close() @@ -1566,9 +1566,9 @@ def test_host_address(): def test_dashboard_address(): pytest.importorskip("bokeh") s = yield Scheduler(dashboard_address="127.0.0.1:8901", port=0) - assert s.services["bokeh"].port == 8901 + assert s.services["dashboard"].port == 8901 yield s.close() s = yield Scheduler(dashboard_address="127.0.0.1", port=0) - assert s.services["bokeh"].port + assert s.services["dashboard"].port yield s.close() diff --git a/distributed/tests/test_worker.py b/distributed/tests/test_worker.py index bf4e483f44..12a6b5ff68 100644 --- a/distributed/tests/test_worker.py +++ b/distributed/tests/test_worker.py @@ -972,25 +972,25 @@ def test_worker_fds(s): @gen_cluster(ncores=[]) def test_service_hosts_match_worker(s): pytest.importorskip("bokeh") - from distributed.bokeh.worker import BokehWorker + from distributed.dashboard import BokehWorker - services = {("bokeh", ":0"): BokehWorker} + services = {("dashboard", ":0"): BokehWorker} - w = Worker(s.address, services={("bokeh", ":0"): BokehWorker}) + w = Worker(s.address, services={("dashboard", ":0"): BokehWorker}) yield w._start("tcp://0.0.0.0") - sock = first(w.services["bokeh"].server._http._sockets.values()) + sock = first(w.services["dashboard"].server._http._sockets.values()) assert sock.getsockname()[0] in ("::", "0.0.0.0") yield w.close() - w = Worker(s.address, services={("bokeh", ":0"): BokehWorker}) + w = Worker(s.address, services={("dashboard", ":0"): BokehWorker}) yield w._start("tcp://127.0.0.1") - sock = first(w.services["bokeh"].server._http._sockets.values()) + sock = first(w.services["dashboard"].server._http._sockets.values()) assert sock.getsockname()[0] in ("::", "0.0.0.0") yield w.close() - w = Worker(s.address, services={("bokeh", 0): BokehWorker}) + w = Worker(s.address, services={("dashboard", 0): BokehWorker}) yield w._start("tcp://127.0.0.1") - sock = first(w.services["bokeh"].server._http._sockets.values()) + sock = first(w.services["dashboard"].server._http._sockets.values()) assert sock.getsockname()[0] == "127.0.0.1" yield w.close() @@ -998,14 +998,14 @@ def test_service_hosts_match_worker(s): @gen_cluster(ncores=[]) def test_start_services(s): pytest.importorskip("bokeh") - from distributed.bokeh.worker import BokehWorker + from distributed.dashboard import BokehWorker - services = {("bokeh", ":1234"): BokehWorker} + services = {("dashboard", ":1234"): BokehWorker} w = Worker(s.address, services=services) yield w._start() - assert w.services["bokeh"].server.port == 1234 + assert w.services["dashboard"].server.port == 1234 yield w.close() diff --git a/distributed/worker.py b/distributed/worker.py index 667bd83490..711dad3165 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -546,13 +546,13 @@ def __init__( if dashboard_address is not None: try: - from distributed.bokeh.worker import BokehWorker + from distributed.dashboard import BokehWorker except ImportError: logger.debug("To start diagnostics web server please install Bokeh") else: - self.service_specs[("bokeh", dashboard_address)] = ( + self.service_specs[("dashboard", dashboard_address)] = ( BokehWorker, - (service_kwargs or {}).get("bokeh", {}), + (service_kwargs or {}).get("dashboard", {}), ) self.metrics = dict(metrics) if metrics else {} diff --git a/setup.py b/setup.py index 3ef26a047d..0df22f3f91 100755 --- a/setup.py +++ b/setup.py @@ -2,46 +2,51 @@ import os from setuptools import setup -import sys import versioneer -requires = open('requirements.txt').read().strip().split('\n') +requires = open("requirements.txt").read().strip().split("\n") install_requires = [] extras_require = {} for r in requires: - if ';' in r: + if ";" in r: # requirements.txt conditional dependencies need to be reformatted for wheels # to the form: `'[extra_name]:condition' : ['requirements']` - req, cond = r.split(';', 1) - cond = ':' + cond + req, cond = r.split(";", 1) + cond = ":" + cond cond_reqs = extras_require.setdefault(cond, []) cond_reqs.append(req) else: install_requires.append(r) -setup(name='distributed', - version=versioneer.get_version(), - cmdclass=versioneer.get_cmdclass(), - description='Distributed scheduler for Dask', - url='https://distributed.readthedocs.io/en/latest/', - maintainer='Matthew Rocklin', - maintainer_email='mrocklin@gmail.com', - license='BSD', - package_data={'': ['templates/index.html', 'template.html'], - 'distributed': ['bokeh/templates/*.html']}, - include_package_data=True, - install_requires=install_requires, - extras_require=extras_require, - packages=['distributed', - 'distributed.bokeh', - 'distributed.cli', - 'distributed.comm', - 'distributed.deploy', - 'distributed.diagnostics', - 'distributed.protocol'], - long_description=(open('README.rst').read() if os.path.exists('README.rst') - else ''), - classifiers=[ +setup( + name="distributed", + version=versioneer.get_version(), + cmdclass=versioneer.get_cmdclass(), + description="Distributed scheduler for Dask", + url="https://distributed.readthedocs.io/en/latest/", + maintainer="Matthew Rocklin", + maintainer_email="mrocklin@gmail.com", + license="BSD", + package_data={ + "": ["templates/index.html", "template.html"], + "distributed": ["dashboard/templates/*.html"], + }, + include_package_data=True, + install_requires=install_requires, + extras_require=extras_require, + packages=[ + "distributed", + "distributed.dashboard", + "distributed.cli", + "distributed.comm", + "distributed.deploy", + "distributed.diagnostics", + "distributed.protocol", + ], + long_description=( + open("README.rst").read() if os.path.exists("README.rst") else "" + ), + classifiers=[ "Development Status :: 5 - Production/Stable", "Intended Audience :: Developers", "Intended Audience :: Science/Research", @@ -54,8 +59,8 @@ "Programming Language :: Python :: 3.7", "Topic :: Scientific/Engineering", "Topic :: System :: Distributed Computing", - ], - entry_points=''' + ], + entry_points=""" [console_scripts] dask-ssh=distributed.cli.dask_ssh:go dask-submit=distributed.cli.dask_submit:go @@ -63,5 +68,6 @@ dask-scheduler=distributed.cli.dask_scheduler:go dask-worker=distributed.cli.dask_worker:go dask-mpi=distributed.cli.dask_mpi:go - ''', - zip_safe=False) + """, + zip_safe=False, +) From 4e3ba76be99ae5d572364e3b8a05a5a7ec42cce5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Est=C3=A8ve?= Date: Wed, 29 May 2019 20:50:27 +0200 Subject: [PATCH 27/43] Add back LocalCluster.__repr__. (#2732) LocalCluster.__repr__ was removed in #2675. --- distributed/deploy/local.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/distributed/deploy/local.py b/distributed/deploy/local.py index 17150fdf70..298c47d7a3 100644 --- a/distributed/deploy/local.py +++ b/distributed/deploy/local.py @@ -197,6 +197,13 @@ def __init__( ) self.scale(n_workers) + def __repr__(self): + return "LocalCluster(%r, workers=%d, ncores=%d)" % ( + self.scheduler_address, + len(self.workers), + sum(w.ncores for w in self.workers.values()), + ) + def nprocesses_nthreads(n): """ From 23b1d93ca7028e0b3dad0b55d6d133559b1d7f35 Mon Sep 17 00:00:00 2001 From: Manuel Garrido Date: Wed, 29 May 2019 22:10:54 +0100 Subject: [PATCH 28/43] add kwargs to progressbars (#2638) * add kwargs to progressbars * remove assertion * linting and add kwarg test for progress bar --- distributed/diagnostics/progressbar.py | 25 ++++++++++++++----- .../diagnostics/tests/test_progressbar.py | 8 ++++++ 2 files changed, 27 insertions(+), 6 deletions(-) diff --git a/distributed/diagnostics/progressbar.py b/distributed/diagnostics/progressbar.py index 08ba8f7da6..8a381562f2 100644 --- a/distributed/diagnostics/progressbar.py +++ b/distributed/diagnostics/progressbar.py @@ -118,6 +118,7 @@ def __init__( loop=None, complete=True, start=True, + **kwargs ): super(TextProgressBar, self).__init__(keys, scheduler, interval, complete) self.width = width @@ -154,7 +155,13 @@ class ProgressWidget(ProgressBar): """ def __init__( - self, keys, scheduler=None, interval="100ms", complete=False, loop=None + self, + keys, + scheduler=None, + interval="100ms", + complete=False, + loop=None, + **kwargs ): super(ProgressWidget, self).__init__(keys, scheduler, interval, complete) @@ -207,7 +214,13 @@ def _draw_bar(self, remaining, all, **kwargs): class MultiProgressBar(object): def __init__( - self, keys, scheduler=None, func=key_split, interval="100ms", complete=False + self, + keys, + scheduler=None, + func=key_split, + interval="100ms", + complete=False, + **kwargs ): self.scheduler = get_scheduler(scheduler) @@ -306,6 +319,7 @@ def __init__( interval=0.1, func=key_split, complete=False, + **kwargs ): super(MultiProgressWidget, self).__init__( keys, scheduler, func, interval, complete @@ -425,7 +439,6 @@ def progress(*futures, **kwargs): notebook = kwargs.pop("notebook", None) multi = kwargs.pop("multi", True) complete = kwargs.pop("complete", True) - assert not kwargs futures = futures_of(futures) if not isinstance(futures, (set, list)): @@ -434,9 +447,9 @@ def progress(*futures, **kwargs): notebook = is_kernel() # often but not always correct assumption if notebook: if multi: - bar = MultiProgressWidget(futures, complete=complete) + bar = MultiProgressWidget(futures, complete=complete, **kwargs) else: - bar = ProgressWidget(futures, complete=complete) + bar = ProgressWidget(futures, complete=complete, **kwargs) return bar else: - TextProgressBar(futures, complete=complete) + TextProgressBar(futures, complete=complete, **kwargs) diff --git a/distributed/diagnostics/tests/test_progressbar.py b/distributed/diagnostics/tests/test_progressbar.py index ac21f1637b..ba42f2ce6e 100644 --- a/distributed/diagnostics/tests/test_progressbar.py +++ b/distributed/diagnostics/tests/test_progressbar.py @@ -76,3 +76,11 @@ def test_progress_function(client, capsys): progress(f) check_bar_completed(capsys) + + +def test_progress_function_w_kwargs(client, capsys): + f = client.submit(lambda: 1) + g = client.submit(lambda: 2) + + progress(f, interval="20ms") + check_bar_completed(capsys) From d9626a59fa0ee5953293666591a083d2c249ddc1 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Thu, 30 May 2019 15:47:16 -0700 Subject: [PATCH 29/43] Close nannies gracefully (#2731) Previously a worker process could be stopped before it told its nanny that it was going away. Now we intentionally tell the nanny ahead of time from the scheduler (and the worker for good measure) before we start the shutdown procedure. --- distributed/deploy/tests/test_local.py | 1 + distributed/nanny.py | 11 ++++++++++- distributed/scheduler.py | 1 + distributed/utils_test.py | 6 ++++++ distributed/worker.py | 7 ++++++- 5 files changed, 24 insertions(+), 2 deletions(-) diff --git a/distributed/deploy/tests/test_local.py b/distributed/deploy/tests/test_local.py index 6f9a4a0324..8aad6675f8 100644 --- a/distributed/deploy/tests/test_local.py +++ b/distributed/deploy/tests/test_local.py @@ -474,6 +474,7 @@ def test_death_timeout_raises(loop): loop=loop, ) as cluster: pass + LocalCluster._instances.clear() # ignore test hygiene checks @pytest.mark.skipif(sys.version_info < (3, 6), reason="Unknown") diff --git a/distributed/nanny.py b/distributed/nanny.py index a27f713ea6..8d2a38192d 100644 --- a/distributed/nanny.py +++ b/distributed/nanny.py @@ -131,6 +131,7 @@ def __init__( "restart": self.restart, # cannot call it 'close' on the rpc side for naming conflict "terminate": self.close, + "close_gracefully": self.close_gracefully, "run": self.run, } @@ -355,7 +356,7 @@ def _on_exit(self, exitcode): return try: - if self.status not in ("closing", "closed"): + if self.status not in ("closing", "closed", "closing-gracefully"): if self.auto_restart: logger.warning("Restarting worker") yield self.instantiate() @@ -372,6 +373,14 @@ def _close(self, *args, **kwargs): warnings.warn("Worker._close has moved to Worker.close", stacklevel=2) return self.close(*args, **kwargs) + def close_gracefully(self, comm=None): + """ + A signal that we shouldn't try to restart workers if they go away + + This is used as part of the cluster shutdown process. + """ + self.status = "closing-gracefully" + @gen.coroutine def close(self, comm=None, timeout=5, report=None): """ diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 9db6477aeb..ca3c1241ea 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -1265,6 +1265,7 @@ def close(self, comm=None, fast=False, close_workers=False): setproctitle("dask-scheduler [closing]") if close_workers: + self.broadcast(msg={"op": "close_gracefully"}, nanny=True) for worker in self.workers: self.worker_send(worker, {"op": "close"}) for i in range(20): # wait a second for send signals to clear diff --git a/distributed/utils_test.py b/distributed/utils_test.py index d61046f2a4..10784c6f75 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -44,6 +44,7 @@ from .comm.utils import offload from .config import initialize_logging from .core import connect, rpc, CommClosedError +from .deploy import SpecCluster from .metrics import time from .process import _cleanup_dangling from .proctitle import enable_proctitle_on_children @@ -1477,6 +1478,7 @@ def check_instances(): Client._instances.clear() Worker._instances.clear() Scheduler._instances.clear() + SpecCluster._instances.clear() # assert all(n.status == "closed" for n in Nanny._instances), { # n: n.status for n in Nanny._instances # } @@ -1514,6 +1516,10 @@ def check_instances(): n: n.status for n in Nanny._instances } + # assert not list(SpecCluster._instances) # TODO + assert all(c.status == "closed" for c in SpecCluster._instances) + SpecCluster._instances.clear() + Nanny._instances.clear() DequeHandler.clear_all_instances() diff --git a/distributed/worker.py b/distributed/worker.py index 711dad3165..d0bc735ec6 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -1007,6 +1007,11 @@ def close(self, report=True, timeout=10, nanny=True, executor_wait=True): except ValueError: # address not available if already closed logger.info("Stopping worker") self.status = "closing" + + if nanny and self.nanny: + with self.rpc(self.nanny) as r: + yield r.close_gracefully() + setproctitle("dask-worker [closing]") yield [ @@ -1015,7 +1020,6 @@ def close(self, report=True, timeout=10, nanny=True, executor_wait=True): if hasattr(plugin, "teardown") ] - self.stop() for pc in self.periodic_callbacks.values(): pc.stop() with ignoring(EnvironmentError, gen.TimeoutError): @@ -1047,6 +1051,7 @@ def close(self, report=True, timeout=10, nanny=True, executor_wait=True): with self.rpc(self.nanny) as r: yield r.terminate() + self.stop() self.rpc.close() self._closed.set() From a8504d6d4a007ea5d427c2d17434b3dd22350e0a Mon Sep 17 00:00:00 2001 From: Benjamin Zaitlen Date: Fri, 31 May 2019 09:51:47 -0400 Subject: [PATCH 30/43] Add Experimental UCX Comm (#2591) --- distributed/cli/dask_scheduler.py | 5 + distributed/cli/dask_worker.py | 5 + distributed/comm/__init__.py | 5 + distributed/comm/addressing.py | 4 +- distributed/comm/tests/__init__.py | 0 distributed/comm/tests/test_comms.py | 14 +- distributed/comm/tests/test_ucx.py | 296 +++++++++++++++++++++++ distributed/comm/ucx.py | 308 ++++++++++++++++++++++++ distributed/core.py | 10 +- distributed/deploy/local.py | 8 +- distributed/preloading.py | 2 + distributed/protocol/__init__.py | 19 ++ distributed/protocol/core.py | 1 + distributed/protocol/cuda.py | 33 +++ distributed/protocol/cudf.py | 74 ++++++ distributed/protocol/cupy.py | 42 ++++ distributed/protocol/numba.py | 61 +++++ distributed/protocol/tests/test_cupy.py | 12 + distributed/protocol/utils.py | 5 +- 19 files changed, 893 insertions(+), 11 deletions(-) create mode 100644 distributed/comm/tests/__init__.py create mode 100644 distributed/comm/tests/test_ucx.py create mode 100644 distributed/comm/ucx.py create mode 100644 distributed/protocol/cuda.py create mode 100644 distributed/protocol/cudf.py create mode 100644 distributed/protocol/cupy.py create mode 100644 distributed/protocol/numba.py create mode 100644 distributed/protocol/tests/test_cupy.py diff --git a/distributed/cli/dask_scheduler.py b/distributed/cli/dask_scheduler.py index 1f78426f63..b27e68eaa9 100755 --- a/distributed/cli/dask_scheduler.py +++ b/distributed/cli/dask_scheduler.py @@ -39,6 +39,9 @@ default=None, help="Preferred network interface like 'eth0' or 'ib0'", ) +@click.option( + "--protocol", type=str, default=None, help="Protocol like tcp, tls, or ucx" +) @click.option( "--tls-ca-file", type=pem_file_option_type, @@ -121,6 +124,7 @@ def main( pid_file, scheduler_file, interface, + protocol, local_directory, preload, preload_argv, @@ -190,6 +194,7 @@ def del_pid_file(): host=host, port=port, interface=interface, + protocol=protocol, dashboard_address=dashboard_address if dashboard else None, service_kwargs={"dashboard": {"prefix": dashboard_prefix}}, ) diff --git a/distributed/cli/dask_worker.py b/distributed/cli/dask_worker.py index c4f83f6140..2cf570cfc1 100755 --- a/distributed/cli/dask_worker.py +++ b/distributed/cli/dask_worker.py @@ -104,6 +104,9 @@ @click.option( "--interface", type=str, default=None, help="Network interface like 'eth0' or 'ib0'" ) +@click.option( + "--protocol", type=str, default=None, help="Protocol like tcp, tls, or ucx" +) @click.option("--nthreads", type=int, default=0, help="Number of threads per process.") @click.option( "--nprocs", @@ -197,6 +200,7 @@ def main( local_directory, scheduler_file, interface, + protocol, death_timeout, preload, preload_argv, @@ -338,6 +342,7 @@ def del_pid_file(): security=sec, contact_address=contact_address, interface=interface, + protocol=protocol, host=host, port=port, dashboard_address=dashboard_address if dashboard else None, diff --git a/distributed/comm/__init__.py b/distributed/comm/__init__.py index 0f7c701847..e0615b38c7 100644 --- a/distributed/comm/__init__.py +++ b/distributed/comm/__init__.py @@ -18,5 +18,10 @@ def _register_transports(): from . import inproc from . import tcp + try: + from . import ucx + except ImportError: + pass + _register_transports() diff --git a/distributed/comm/addressing.py b/distributed/comm/addressing.py index 3d79befe0f..d707adb84a 100644 --- a/distributed/comm/addressing.py +++ b/distributed/comm/addressing.py @@ -72,6 +72,8 @@ def _default(): raise ValueError("missing port number in address %r" % (address,)) return default_port + if "://" in address: + _, address = address.split("://") if address.startswith("["): # IPv6 notation: '[addr]:port' or '[addr]'. # The address may contain multiple colons. @@ -101,7 +103,7 @@ def unparse_host_port(host, port=None): """ if ":" in host and not host.startswith("["): host = "[%s]" % host - if port: + if port is not None: return "%s:%s" % (host, port) else: return host diff --git a/distributed/comm/tests/__init__.py b/distributed/comm/tests/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/distributed/comm/tests/test_comms.py b/distributed/comm/tests/test_comms.py index 0e8782718a..e761deeab8 100644 --- a/distributed/comm/tests/test_comms.py +++ b/distributed/comm/tests/test_comms.py @@ -26,6 +26,7 @@ from distributed.protocol import to_serialize, Serialized, serialize, deserialize +from distributed.comm.registry import backends from distributed.comm import ( tcp, inproc, @@ -40,7 +41,6 @@ get_local_address_for, ) - EXTERNAL_IP4 = get_ip() if has_ipv6(): with warnings.catch_warnings(record=True): @@ -154,7 +154,6 @@ def test_unparse_host_port(): assert f("[::1]", 123) == "[::1]:123" assert f("127.0.0.1") == "127.0.0.1" - assert f("127.0.0.1", 0) == "127.0.0.1" assert f("127.0.0.1", None) == "127.0.0.1" assert f("127.0.0.1", "*") == "127.0.0.1:*" @@ -488,7 +487,7 @@ def handle_comm(comm): # Check listener properties bound_addr = listener.listen_address bound_scheme, bound_loc = parse_address(bound_addr) - assert bound_scheme in ("inproc", "tcp", "tls") + assert bound_scheme in backends assert bound_scheme == parse_address(addr)[0] if check_listen_addr is not None: @@ -530,6 +529,15 @@ def client_communicate(key, delay=0): listener.stop() +@gen_test() +def test_ucx_client_server(): + pytest.importorskip("distributed.comm.ucx") + import ucp + + addr = ucp.get_address() + yield check_client_server("ucx://" + addr) + + def tcp_eq(expected_host, expected_port=None): def checker(loc): host, port = parse_host_port(loc) diff --git a/distributed/comm/tests/test_ucx.py b/distributed/comm/tests/test_ucx.py new file mode 100644 index 0000000000..55a2f4ec82 --- /dev/null +++ b/distributed/comm/tests/test_ucx.py @@ -0,0 +1,296 @@ +import asyncio + +import pytest + +ucp = pytest.importorskip("ucp") + +from distributed import Client +from distributed.comm import ucx, listen, connect +from distributed.comm.registry import backends, get_backend +from distributed.comm import ucx, parse_address +from distributed.protocol import to_serialize +from distributed.deploy.local import LocalCluster +from distributed.utils_test import gen_test, loop, inc # noqa: 401 + +from .test_comms import check_deserialize + + +HOST = ucp.get_address() + + +def test_registered(): + assert "ucx" in backends + backend = get_backend("ucx") + assert isinstance(backend, ucx.UCXBackend) + + +async def get_comm_pair( + listen_addr="ucx://" + HOST, listen_args=None, connect_args=None, **kwargs +): + q = asyncio.queues.Queue() + + async def handle_comm(comm): + await q.put(comm) + + # Workaround for hanging test in + # pytest distributed/comm/tests/test_ucx.py::test_comm_objs -vs --count=2 + # on the second time through. + ucp._libs.ucp_py.reader_added = 0 + + listener = listen(listen_addr, handle_comm, connection_args=listen_args, **kwargs) + with listener: + comm = await connect( + listener.contact_address, connection_args=connect_args, **kwargs + ) + serv_com = await q.get() + return comm, serv_com + + +@pytest.mark.asyncio +async def test_ping_pong(): + com, serv_com = await get_comm_pair() + msg = {"op": "ping"} + await com.write(msg) + result = await serv_com.read() + assert result == msg + result["op"] = "pong" + + await serv_com.write(result) + + result = await com.read() + assert result == {"op": "pong"} + + await com.close() + await serv_com.close() + + +@pytest.mark.asyncio +async def test_comm_objs(): + comm, serv_comm = await get_comm_pair() + + scheme, loc = parse_address(comm.peer_address) + assert scheme == "ucx" + + scheme, loc = parse_address(serv_comm.peer_address) + assert scheme == "ucx" + + assert comm.peer_address == serv_comm.local_address + + +def test_ucx_specific(): + """ + Test concrete UCX API. + """ + # TODO: + # 1. ensure exceptions in handle_comm fail the test + # 2. Use dict in read / write, put seralization there. + # 3. Test peer_address + # 4. Test cleanup + async def f(): + address = "ucx://{}:{}".format(HOST, 0) + + async def handle_comm(comm): + msg = await comm.read() + msg["op"] = "pong" + await comm.write(msg) + assert comm.closed() is False + await comm.close() + assert comm.closed + + listener = ucx.UCXListener(address, handle_comm) + listener.start() + host, port = listener.get_host_port() + assert host.count(".") == 3 + assert port > 0 + + connector = ucx.UCXConnector() + l = [] + + async def client_communicate(key, delay=0): + addr = "%s:%d" % (host, port) + comm = await connector.connect(addr) + # TODO: peer_address + # assert comm.peer_address == 'ucx://' + addr + assert comm.extra_info == {} + msg = {"op": "ping", "data": key} + await comm.write(msg) + if delay: + await asyncio.sleep(delay) + msg = await comm.read() + assert msg == {"op": "pong", "data": key} + l.append(key) + return comm + assert comm.closed() is False + await comm.close() + assert comm.closed + + comm = await client_communicate(key=1234, delay=0.5) + + # Many clients at once + N = 2 + futures = [client_communicate(key=i, delay=0.05) for i in range(N)] + await asyncio.gather(*futures) + assert set(l) == {1234} | set(range(N)) + + asyncio.run(f()) + + +@pytest.mark.asyncio +async def test_ping_pong_data(): + np = pytest.importorskip("numpy") + + data = np.ones((10, 10)) + + com, serv_com = await get_comm_pair() + msg = {"op": "ping", "data": to_serialize(data)} + await com.write(msg) + result = await serv_com.read() + result["op"] = "pong" + data2 = result.pop("data") + np.testing.assert_array_equal(data2, data) + + await serv_com.write(result) + + result = await com.read() + assert result == {"op": "pong"} + + await com.close() + await serv_com.close() + + +@gen_test() +def test_ucx_deserialize(): + yield check_deserialize("tcp://") + + +@pytest.mark.asyncio +async def test_ping_pong_cudf(): + # if this test appears after cupy an import error arises + # *** ImportError: /usr/lib/x86_64-linux-gnu/libstdc++.so.6: version `CXXABI_1.3.11' + # not found (required by python3.7/site-packages/pyarrow/../../../libarrow.so.12) + cudf = pytest.importorskip("cudf") + + df = cudf.DataFrame({"A": [1, 2, None], "B": [1.0, 2.0, None]}) + + com, serv_com = await get_comm_pair() + msg = {"op": "ping", "data": to_serialize(df)} + + await com.write(msg) + result = await serv_com.read() + data2 = result.pop("data") + assert result["op"] == "ping" + + +@pytest.mark.asyncio +@pytest.mark.parametrize("shape", [(100,), (10, 10), (4947,)]) +async def test_ping_pong_cupy(shape): + cupy = pytest.importorskip("cupy") + com, serv_com = await get_comm_pair() + + arr = cupy.random.random(shape) + msg = {"op": "ping", "data": to_serialize(arr)} + + _, result = await asyncio.gather(com.write(msg), serv_com.read()) + data2 = result.pop("data") + + assert result["op"] == "ping" + cupy.testing.assert_array_equal(arr, data2) + await com.close() + await serv_com.close() + + +@pytest.mark.slow +@pytest.mark.asyncio +@pytest.mark.parametrize( + "n", + [ + int(1e9), + pytest.param( + int(2.5e9), marks=[pytest.mark.xfail(reason="integer type in ucx-py")] + ), + ], +) +async def test_large_cupy(n): + cupy = pytest.importorskip("cupy") + com, serv_com = await get_comm_pair() + + arr = cupy.ones(n, dtype="u1") + msg = {"op": "ping", "data": to_serialize(arr)} + + _, result = await asyncio.gather(com.write(msg), serv_com.read()) + data2 = result.pop("data") + + assert result["op"] == "ping" + assert len(data2) == len(arr) + await com.close() + await serv_com.close() + + +@pytest.mark.asyncio +async def test_ping_pong_numba(): + np = pytest.importorskip("numpy") + numba = pytest.importorskip("numba") + import numba.cuda + + arr = np.arange(10) + arr = numba.cuda.to_device(arr) + + com, serv_com = await get_comm_pair() + msg = {"op": "ping", "data": to_serialize(arr)} + + await com.write(msg) + result = await serv_com.read() + data2 = result.pop("data") + assert result["op"] == "ping" + + +@pytest.mark.skip(reason="hangs") +@pytest.mark.parametrize("processes", [True, False]) +def test_ucx_localcluster(loop, processes): + if processes: + kwargs = {"env": {"UCX_MEMTYPE_CACHE": "n"}} + else: + kwargs = {} + + ucx_addr = ucp.get_address() + with LocalCluster( + protocol="ucx", + interface="ib0", + dashboard_address=None, + n_workers=2, + threads_per_worker=1, + processes=processes, + loop=loop, + **kwargs, + ) as cluster: + with Client(cluster) as client: + x = client.submit(inc, 1) + x.result() + assert x.key in cluster.scheduler.tasks + if not processes: + assert any(w.data == {x.key: 2} for w in cluster.workers.values()) + assert len(cluster.scheduler.workers) == 2 + + +def test_tcp_localcluster(loop): + ucx_addr = "127.0.0.1" + port = 13337 + env = {"UCX_MEMTYPE_CACHE": "n"} + with LocalCluster( + 2, + scheduler_port=port, + ip=ucx_addr, + processes=True, + threads_per_worker=1, + dashboard_address=None, + silence_logs=False, + env=env, + ) as cluster: + pass + # with Client(cluster) as e: + # x = e.submit(inc, 1) + # x.result() + # assert x.key in c.scheduler.tasks + # assert any(w.data == {x.key: 2} for w in c.workers) + # assert e.loop is c.loop + # print(c.scheduler.workers) diff --git a/distributed/comm/ucx.py b/distributed/comm/ucx.py new file mode 100644 index 0000000000..3f3f0bfe94 --- /dev/null +++ b/distributed/comm/ucx.py @@ -0,0 +1,308 @@ +""" +:ref:`UCX`_ based communications for distributed. + +See :ref:`communications` for more. + +.. _UCX: https://github.com/openucx/ucx +""" +import asyncio +import logging +import struct + +from .addressing import parse_host_port, unparse_host_port +from .core import Comm, Connector, Listener, CommClosedError +from .registry import Backend, backends +from .utils import ensure_concrete_host, to_frames, from_frames +from ..utils import ensure_ip, get_ip, get_ipv6, nbytes + +import ucp + +import os + +os.environ.setdefault("UCX_RNDV_SCHEME", "put_zcopy") +os.environ.setdefault("UCX_MEMTYPE_CACHE", "n") +os.environ.setdefault("UCX_TLS", "rc,cuda_copy") + +logger = logging.getLogger(__name__) +MAX_MSG_LOG = 23 + + +# ---------------------------------------------------------------------------- +# Comm Interface +# ---------------------------------------------------------------------------- + + +class UCX(Comm): + """Comm object using UCP. + + Parameters + ---------- + ep : ucp.Endpoint + The UCP endpoint. + address : str + The address, prefixed with `ucx://` to use. + deserialize : bool, default True + Whether to deserialize data in :meth:`distributed.protocol.loads` + + Notes + ----- + The read-write cycle uses the following pattern: + + Each msg is serialized into a number of "data" frames. We prepend these + real frames with two additional frames + + 1. is_gpus: Boolean indicator for whether the frame should be + received into GPU memory. Packed in '?' format. Unpack with + ``?`` format. + 2. frame_size : Unsigned int describing the size of frame (in bytes) + to receive. Packed in 'Q' format, so a length-0 frame is equivalent + to an unsized frame. Unpacked with ``Q``. + + The expected read cycle is + + 1. Read the frame describing number of frames + 2. Read the frame describing whether each data frame is gpu-bound + 3. Read the frame describing whether each data frame is sized + 4. Read all the data frames. + """ + + def __init__( + self, ep: ucp.Endpoint, local_addr: str, peer_addr: str, deserialize=True + ): + Comm.__init__(self) + self._ep = ep + if local_addr: + assert local_addr.startswith("ucx") + assert peer_addr.startswith("ucx") + self._local_addr = local_addr + self._peer_addr = peer_addr + self.deserialize = deserialize + self.comm_flag = None + logger.debug("UCX.__init__ %s", self) + + @property + def local_address(self) -> str: + return self._local_addr + + @property + def peer_address(self) -> str: + return self._peer_addr + + async def write( + self, + msg: dict, + serializers=("cuda", "dask", "pickle", "error"), + on_error: str = "message", + ): + if serializers is None: + serializers = ("cuda", "dask", "pickle", "error") + # msg can also be a list of dicts when sending batched messages + frames = await to_frames(msg, serializers=serializers, on_error=on_error) + is_gpus = b"".join( + [ + struct.pack("?", hasattr(frame, "__cuda_array_interface__")) + for frame in frames + ] + ) + sizes = b"".join([struct.pack("Q", nbytes(frame)) for frame in frames]) + + nframes = struct.pack("Q", len(frames)) + + meta = b"".join([nframes, is_gpus, sizes]) + + await self.ep.send_obj(meta) + + for frame in frames: + await self.ep.send_obj(frame) + return sum(map(nbytes, frames)) + + async def read(self, deserializers=("cuda", "dask", "pickle", "error")): + if deserializers is None: + deserializers = ("cuda", "dask", "pickle", "error") + resp = await self.ep.recv_future() + obj = ucp.get_obj_from_msg(resp) + nframes, = struct.unpack("Q", obj[:8]) # first eight bytes for number of frames + + gpu_frame_msg = obj[ + 8 : 8 + nframes + ] # next nframes bytes for if they're GPU frames + is_gpus = struct.unpack("{}?".format(nframes), gpu_frame_msg) + + sized_frame_msg = obj[8 + nframes :] # then the rest for frame sizes + sizes = struct.unpack("{}Q".format(nframes), sized_frame_msg) + + frames = [] + + for i, (is_gpu, size) in enumerate(zip(is_gpus, sizes)): + if size > 0: + resp = await self.ep.recv_obj(size, cuda=is_gpu) + else: + resp = await self.ep.recv_future() + frame = ucp.get_obj_from_msg(resp) + frames.append(frame) + + msg = await from_frames( + frames, deserialize=self.deserialize, deserializers=deserializers + ) + + return msg + + def abort(self): + if self._ep: + ucp.destroy_ep(self._ep) + logger.debug("Destroyed UCX endpoint") + self._ep = None + + @property + def ep(self): + if self._ep: + return self._ep + else: + raise CommClosedError("UCX Endpoint is closed") + + async def close(self): + # TODO: Handle in-flight messages? + # sleep is currently used to help flush buffer + self.abort() + + def closed(self): + return self._ep is None + + +class UCXConnector(Connector): + prefix = "ucx://" + comm_class = UCX + encrypted = False + + async def connect(self, address: str, deserialize=True, **connection_args) -> UCX: + logger.debug("UCXConnector.connect: %s", address) + ucp.init() + ip, port = parse_host_port(address) + ep = await ucp.get_endpoint(ip.encode(), port) + return self.comm_class( + ep, + local_addr=None, + peer_addr=self.prefix + address, + deserialize=deserialize, + ) + + +class UCXListener(Listener): + # MAX_LISTENERS 256 in ucx-py + prefix = UCXConnector.prefix + comm_class = UCXConnector.comm_class + encrypted = UCXConnector.encrypted + + def __init__( + self, address: str, comm_handler: None, deserialize=False, **connection_args + ): + if not address.startswith("ucx"): + address = "ucx://" + address + self.ip, self._input_port = parse_host_port(address, default_port=0) + self.comm_handler = comm_handler + self.deserialize = deserialize + self._ep = None # type: ucp.Endpoint + self.listener_instance = None # type: ucp.ListenerFuture + self.ucp_server = None + self._task = None + + self.connection_args = connection_args + self._task = None + + @property + def port(self): + return self.ucp_server.port + + @property + def address(self): + return "ucx://" + self.ip + ":" + str(self.port) + + def start(self): + async def serve_forever(client_ep, listener_instance): + ucx = UCX( + client_ep, + local_addr=self.address, + peer_addr=self.address, # TODO: https://github.com/Akshay-Venkatesh/ucx-py/issues/111 + deserialize=self.deserialize, + ) + self.listener_instance = listener_instance + if self.comm_handler: + await self.comm_handler(ucx) + + ucp.init() + self.ucp_server = ucp.start_listener( + serve_forever, listener_port=self._input_port, is_coroutine=True + ) + + try: + loop = asyncio.get_running_loop() + except (RuntimeError, AttributeError): + loop = asyncio.get_event_loop() + + t = loop.create_task(self.ucp_server.coroutine) + self._task = t + + def stop(self): + # What all should this do? + if self._task: + self._task.cancel() + + if self._ep: + ucp.destroy_ep(self._ep) + # if self.listener_instance: + # ucp.stop_listener(self.listener_instance) + + def get_host_port(self): + # TODO: TCP raises if this hasn't started yet. + return self.ip, self.port + + @property + def listen_address(self): + return self.prefix + unparse_host_port(*self.get_host_port()) + + @property + def contact_address(self): + host, port = self.get_host_port() + host = ensure_concrete_host(host) # TODO: ensure_concrete_host + return self.prefix + unparse_host_port(host, port) + + @property + def bound_address(self): + # TODO: Does this become part of the base API? Kinda hazy, since + # we exclude in for inproc. + return self.get_host_port() + + +class UCXBackend(Backend): + # I / O + + def get_connector(self): + return UCXConnector() + + def get_listener(self, loc, handle_comm, deserialize, **connection_args): + return UCXListener(loc, handle_comm, deserialize, **connection_args) + + # Address handling + # This duplicates BaseTCPBackend + + def get_address_host(self, loc): + return parse_host_port(loc)[0] + + def get_address_host_port(self, loc): + return parse_host_port(loc) + + def resolve_address(self, loc): + host, port = parse_host_port(loc) + return unparse_host_port(ensure_ip(host), port) + + def get_local_address_for(self, loc): + host, port = parse_host_port(loc) + host = ensure_ip(host) + if ":" in host: + local_host = get_ipv6(host) + else: + local_host = get_ip(host) + return unparse_host_port(local_host, None) + + +backends["ucx"] = UCXBackend() diff --git a/distributed/core.py b/distributed/core.py index 17685c9d2d..79c726eed6 100644 --- a/distributed/core.py +++ b/distributed/core.py @@ -484,7 +484,7 @@ def handle_stream(self, comm, extra=None, every_cycle=[]): pdb.set_trace() raise finally: - comm.close() # TODO: why do we need this now? + yield comm.close() assert comm.closed() @gen.coroutine @@ -498,7 +498,7 @@ def close(self): break else: yield gen.sleep(0.05) - yield [comm.close() for comm in self._comms] + yield [comm.close() for comm in self._comms] # then forcefully close for cb in self._ongoing_coroutines: cb.cancel() for i in range(10): @@ -901,7 +901,7 @@ def collect(self): ) for addr, comms in self.available.items(): for comm in comms: - comm.close() + IOLoop.current().add_callback(comm.close) comms.clear() if self.open < self.limit: self.event.set() @@ -914,11 +914,11 @@ def remove(self, addr): if addr in self.available: comms = self.available.pop(addr) for comm in comms: - comm.close() + IOLoop.current().add_callback(comm.close) if addr in self.occupied: comms = self.occupied.pop(addr) for comm in comms: - comm.close() + IOLoop.current().add_callback(comm.close) if self.open < self.limit: self.event.set() diff --git a/distributed/deploy/local.py b/distributed/deploy/local.py index 298c47d7a3..95f178c7c2 100644 --- a/distributed/deploy/local.py +++ b/distributed/deploy/local.py @@ -195,7 +195,13 @@ def __init__( asynchronous=asynchronous, silence_logs=silence_logs, ) - self.scale(n_workers) + + def __repr__(self): + return "LocalCluster(%r, workers=%d, ncores=%d)" % ( + self.scheduler_address, + len(self.workers), + sum(w.ncores for w in self.workers.values()), + ) def __repr__(self): return "LocalCluster(%r, workers=%d, ncores=%d)" % ( diff --git a/distributed/preloading.py b/distributed/preloading.py index 0f08f60f71..a5e67c1611 100644 --- a/distributed/preloading.py +++ b/distributed/preloading.py @@ -100,6 +100,7 @@ def _import_modules(names, file_dir=None): import_module(name) module = sys.modules[name] + logger.info("Import preload module: %s", name) result_modules[name] = { attrname: getattr(module, attrname, None) for attrname in ("dask_setup", "dask_teardown") @@ -137,6 +138,7 @@ def preload_modules(names, parameter=None, file_dir=None, argv=None): dask_setup.callback(parameter, *context.args, **context.params) else: dask_setup(parameter) + logger.info("Run preload setup function: %s", name) if interface["dask_teardown"]: atexit.register(interface["dask_teardown"], parameter) diff --git a/distributed/protocol/__init__.py b/distributed/protocol/__init__.py index 04691ce605..3f98436f4b 100644 --- a/distributed/protocol/__init__.py +++ b/distributed/protocol/__init__.py @@ -4,6 +4,7 @@ from .compression import compressions, default_compression from .core import dumps, loads, maybe_compress, decompress, msgpack +from .cuda import cuda_serialize, cuda_deserialize from .serialize import ( serialize, deserialize, @@ -66,3 +67,21 @@ def _register_arrow(): @dask_deserialize.register_lazy("torchvision") def _register_torch(): from . import torch + + +@cuda_serialize.register_lazy("cupy") +@cuda_deserialize.register_lazy("cupy") +def _register_cupy(): + from . import cupy + + +@cuda_serialize.register_lazy("numba") +@cuda_deserialize.register_lazy("numba") +def _register_numba(): + from . import numba + + +@cuda_serialize.register_lazy("cudf") +@cuda_deserialize.register_lazy("cudf") +def _register_cudf(): + from . import cudf diff --git a/distributed/protocol/core.py b/distributed/protocol/core.py index c1b62b2491..d54dd2e533 100644 --- a/distributed/protocol/core.py +++ b/distributed/protocol/core.py @@ -176,6 +176,7 @@ def loads_msgpack(header, payload): See Also: dumps_msgpack """ + header = bytes(header) if header: header = msgpack.loads(header, use_list=False, **msgpack_opts) else: diff --git a/distributed/protocol/cuda.py b/distributed/protocol/cuda.py new file mode 100644 index 0000000000..13be1d75bb --- /dev/null +++ b/distributed/protocol/cuda.py @@ -0,0 +1,33 @@ +import dask + +from . import pickle +from .serialize import register_serialization_family +from dask.utils import typename + +cuda_serialize = dask.utils.Dispatch("cuda_serialize") +cuda_deserialize = dask.utils.Dispatch("cuda_deserialize") + + +def cuda_dumps(x): + type_name = typename(type(x)) + try: + dumps = cuda_serialize.dispatch(type(x)) + except TypeError: + raise NotImplementedError(type_name) + + header, frames = dumps(x) + + header["type"] = type_name + header["type-serialized"] = pickle.dumps(type(x)) + header["serializer"] = "cuda" + header["compression"] = (None,) * len(frames) # no compression for gpu data + return header, frames + + +def cuda_loads(header, frames): + typ = pickle.loads(header["type-serialized"]) + loads = cuda_deserialize.dispatch(typ) + return loads(header, frames) + + +register_serialization_family("cuda", cuda_dumps, cuda_loads) diff --git a/distributed/protocol/cudf.py b/distributed/protocol/cudf.py new file mode 100644 index 0000000000..018596b156 --- /dev/null +++ b/distributed/protocol/cudf.py @@ -0,0 +1,74 @@ +import cudf +from .cuda import cuda_serialize, cuda_deserialize +from .numba import serialize_numba_ndarray, deserialize_numba_ndarray + + +# TODO: +# 1. Just use positions +# a. Fixes duplicate columns +# b. Fixes non-msgpack-serializable names +# 2. cudf.Series +# 3. Serialize the index + + +@cuda_serialize.register(cudf.DataFrame) +def serialize_cudf_dataframe(x): + sub_headers = [] + arrays = [] + null_masks = [] + null_headers = [] + null_counts = {} + + for label, col in x.iteritems(): + header, [frame] = serialize_numba_ndarray(col.data.mem) + header["name"] = label + sub_headers.append(header) + arrays.append(frame) + if col.null_count: + header, [frame] = serialize_numba_ndarray(col.nullmask.mem) + header["name"] = label + null_headers.append(header) + null_masks.append(frame) + null_counts[label] = col.null_count + + arrays.extend(null_masks) + + header = { + "is_cuda": len(arrays), + "subheaders": sub_headers, + # TODO: the header must be msgpack (de)serializable. + # See if we can avoid names, and just use integer positions. + "columns": x.columns.tolist(), + "null_counts": null_counts, + "null_subheaders": null_headers, + } + + return header, arrays + + +@cuda_deserialize.register(cudf.DataFrame) +def serialize_cudf_dataframe(header, frames): + columns = header["columns"] + n_columns = len(header["columns"]) + n_masks = len(header["null_subheaders"]) + + masks = {} + pairs = [] + + for i in range(n_masks): + subheader = header["null_subheaders"][i] + frame = frames[n_columns + i] + mask = deserialize_numba_ndarray(subheader, [frame]) + masks[subheader["name"]] = mask + + for subheader, frame in zip(header["subheaders"], frames[:n_columns]): + name = subheader["name"] + array = deserialize_numba_ndarray(subheader, [frame]) + + if name in masks: + series = cudf.Series.from_masked_array(array, masks[name]) + else: + series = cudf.Series(array) + pairs.append((name, series)) + + return cudf.DataFrame(pairs) diff --git a/distributed/protocol/cupy.py b/distributed/protocol/cupy.py new file mode 100644 index 0000000000..13c0348a82 --- /dev/null +++ b/distributed/protocol/cupy.py @@ -0,0 +1,42 @@ +""" +Efficient serialization GPU arrays. +""" +import cupy +from .cuda import cuda_serialize, cuda_deserialize + + +@cuda_serialize.register(cupy.ndarray) +def serialize_cupy_ndarray(x): + # TODO: handle non-contiguous + # TODO: Handle order='K' ravel + # TODO: 0d + + if x.flags.c_contiguous or x.flags.f_contiguous: + strides = x.strides + data = x.ravel() # order='K' + else: + x = cupy.ascontiguousarray(x) + strides = x.strides + data = x.ravel() + + dtype = (0, x.dtype.str) + + # used in the ucx comms for gpu/cpu message passing + # 'lengths' set by dask + header = x.__cuda_array_interface__.copy() + header["is_cuda"] = 1 + header["dtype"] = dtype + return header, [data] + + +@cuda_deserialize.register(cupy.ndarray) +def deserialize_cupy_array(header, frames): + frame, = frames + # TODO: put this in ucx... as a kind of "fixup" + try: + frame.typestr = header["typestr"] + frame.shape = header["shape"] + except AttributeError: + pass + arr = cupy.asarray(frame) + return arr diff --git a/distributed/protocol/numba.py b/distributed/protocol/numba.py new file mode 100644 index 0000000000..18405ffebe --- /dev/null +++ b/distributed/protocol/numba.py @@ -0,0 +1,61 @@ +import numba.cuda +from .cuda import cuda_serialize, cuda_deserialize + + +@cuda_serialize.register(numba.cuda.devicearray.DeviceNDArray) +def serialize_numba_ndarray(x): + # TODO: handle non-contiguous + # TODO: handle 2d + # TODO: 0d + + if x.flags["C_CONTIGUOUS"] or x.flags["F_CONTIGUOUS"]: + strides = x.strides + if x.ndim > 1: + data = x.ravel() # order='K' + else: + data = x + else: + raise ValueError("Array must be contiguous") + x = numba.ascontiguousarray(x) + strides = x.strides + if x.ndim > 1: + data = x.ravel() + else: + data = x + + dtype = (0, x.dtype.str) + nbytes = data.dtype.itemsize * data.size + + # used in the ucx comms for gpu/cpu message passing + # 'lengths' set by dask + header = x.__cuda_array_interface__.copy() + header["is_cuda"] = 1 + header["dtype"] = dtype + return header, [data] + + +@cuda_deserialize.register(numba.cuda.devicearray.DeviceNDArray) +def deserialize_numba_ndarray(header, frames): + frame, = frames + # TODO: put this in ucx... as a kind of "fixup" + if isinstance(frame, bytes): + import numpy as np + + arr2 = np.frombuffer(frame, header["typestr"]) + return numba.cuda.to_device(arr2) + + frame.typestr = header["typestr"] + frame.shape = header["shape"] + + # numba & cupy don't properly roundtrip length-zero arrays. + if frame.shape[0] == 0: + arr = numba.cuda.device_array( + header["shape"], + header["typestr"] + # strides? + # order? + ) + return arr + + arr = numba.cuda.as_cuda_array(frame) + return arr diff --git a/distributed/protocol/tests/test_cupy.py b/distributed/protocol/tests/test_cupy.py new file mode 100644 index 0000000000..26940597f8 --- /dev/null +++ b/distributed/protocol/tests/test_cupy.py @@ -0,0 +1,12 @@ +from distributed.protocol import serialize, deserialize +import pytest + +cupy = pytest.importorskip("cupy") + + +def test_serialize_cupy(): + x = cupy.arange(100) + header, frames = serialize(x, serializers=("cuda", "dask", "pickle")) + y = deserialize(header, frames, deserializers=("cuda", "dask", "pickle", "error")) + + assert (x == y).all() diff --git a/distributed/protocol/utils.py b/distributed/protocol/utils.py index 208caebb92..caf4bb8833 100644 --- a/distributed/protocol/utils.py +++ b/distributed/protocol/utils.py @@ -90,7 +90,10 @@ def merge_frames(header, frames): L.append(mv[:l]) frames.append(mv[l:]) l = 0 - out.append(b"".join(map(ensure_bytes, L))) + if len(L) == 1: # no work necessary + out.extend(L) + else: + out.append(b"".join(map(ensure_bytes, L))) return out From 7c3b4d1c59b74b39ccfc0f579ce8295713bfb15a Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Fri, 31 May 2019 11:04:13 -0700 Subject: [PATCH 31/43] Pin pytest >=4 with pip in appveyor and python 3.5 (#2737) --- README.rst | 1 + continuous_integration/setup_conda_environment.cmd | 2 +- continuous_integration/travis/install.sh | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/README.rst b/README.rst index 2d2d285be2..b6f0edd604 100644 --- a/README.rst +++ b/README.rst @@ -3,4 +3,5 @@ Distributed A library for distributed computation. See documentation_ for more details. + .. _documentation: https://distributed.readthedocs.io/en/latest diff --git a/continuous_integration/setup_conda_environment.cmd b/continuous_integration/setup_conda_environment.cmd index 5748a8cf20..3df89fa85f 100644 --- a/continuous_integration/setup_conda_environment.cmd +++ b/continuous_integration/setup_conda_environment.cmd @@ -50,7 +50,7 @@ call activate %CONDA_ENV% %PIP_INSTALL% git+https://github.com/joblib/joblib.git --upgrade %PIP_INSTALL% git+https://github.com/dask/zict --upgrade -%PIP_INSTALL% pytest-repeat pytest-timeout pytest-faulthandler sortedcollections pytest-asyncio +%PIP_INSTALL% "pytest>=4" pytest-repeat pytest-timeout pytest-faulthandler sortedcollections pytest-asyncio @rem Display final environment (for reproducing) %CONDA% list diff --git a/continuous_integration/travis/install.sh b/continuous_integration/travis/install.sh index f1ff25a9bf..cb2dbdf5c8 100644 --- a/continuous_integration/travis/install.sh +++ b/continuous_integration/travis/install.sh @@ -59,7 +59,7 @@ conda install -q \ conda install -c defaults -c conda-forge libunwind conda install --no-deps -c defaults -c numba -c conda-forge stacktrace -pip install -q pytest-repeat pytest-faulthandler pytest-asyncio +pip install -q "pytest>=4" pytest-repeat pytest-faulthandler pytest-asyncio pip install -q git+https://github.com/dask/dask.git --upgrade --no-deps pip install -q git+https://github.com/joblib/joblib.git --upgrade --no-deps From a16b8ff071938c08f64e78bb04636c3b4d619325 Mon Sep 17 00:00:00 2001 From: Caleb Date: Fri, 31 May 2019 16:03:39 -0700 Subject: [PATCH 32/43] Allow user to configure whether workers are daemon. (#2739) Closes #2718 --- .gitignore | 3 +++ distributed/distributed.yaml | 1 + distributed/nanny.py | 2 +- distributed/process.py | 2 +- distributed/tests/test_nanny.py | 35 +++++++++++++++++++++++++++++++++ 5 files changed, 41 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index a3a40e1928..2d70b7ebd7 100644 --- a/.gitignore +++ b/.gitignore @@ -13,3 +13,6 @@ continuous_integration/hdfs-initialized .pytest_cache/ dask-worker-space/ .vscode/ +*.swp +.ycm_extra_conf.py +tags diff --git a/distributed/distributed.yaml b/distributed/distributed.yaml index 3ae9b7ee69..4d78a698e6 100644 --- a/distributed/distributed.yaml +++ b/distributed/distributed.yaml @@ -32,6 +32,7 @@ distributed: incoming: 10 preload: [] preload-argv: [] + daemon: True profile: interval: 10ms # Time between statistical profiling queries diff --git a/distributed/nanny.py b/distributed/nanny.py index 8d2a38192d..59a8083e83 100644 --- a/distributed/nanny.py +++ b/distributed/nanny.py @@ -464,7 +464,7 @@ def start(self): env=self.env, ), ) - self.process.daemon = True + self.process.daemon = dask.config.get("distributed.worker.daemon", default=True) self.process.set_exit_callback(self._on_exit) self.running = Event() self.stopped = Event() diff --git a/distributed/process.py b/distributed/process.py index 5dd9368fdc..556edae290 100644 --- a/distributed/process.py +++ b/distributed/process.py @@ -330,7 +330,7 @@ def daemon(self, value): @atexit.register def _cleanup_dangling(): for proc in list(_dangling): - if proc.daemon and proc.is_alive(): + if proc.is_alive(): try: logger.warning("reaping stray process %s" % (proc,)) proc.terminate() diff --git a/distributed/tests/test_nanny.py b/distributed/tests/test_nanny.py index be0a05afc2..1357a3679e 100644 --- a/distributed/tests/test_nanny.py +++ b/distributed/tests/test_nanny.py @@ -5,6 +5,7 @@ import os import random import sys +import multiprocessing as mp import numpy as np @@ -344,3 +345,37 @@ def test_data_types(c, s): r = yield c.run(lambda dask_worker: type(dask_worker.data)) assert r[w.worker_address] == dict yield w.close() + + +def _noop(x): + """Define here because closures aren't pickleable.""" + pass + + +@gen_cluster( + ncores=[("127.0.0.1", 1)], + client=True, + Worker=Nanny, + config={"distributed.worker.daemon": False}, +) +def test_mp_process_worker_no_daemon(c, s, a): + def multiprocessing_worker(): + p = mp.Process(target=_noop, args=(None,)) + p.start() + p.join() + + yield c.submit(multiprocessing_worker) + + +@gen_cluster( + ncores=[("127.0.0.1", 1)], + client=True, + Worker=Nanny, + config={"distributed.worker.daemon": False}, +) +def test_mp_pool_worker_no_daemon(c, s, a): + def pool_worker(world_size): + with mp.Pool(processes=world_size) as p: + p.map(_noop, range(world_size)) + + yield c.submit(pool_worker, 4) From 861536ca2cbb6039ae9325c672f1b85f8124bc25 Mon Sep 17 00:00:00 2001 From: Michael Spiegel Date: Mon, 3 Jun 2019 17:37:41 +0200 Subject: [PATCH 33/43] Fix the resource key representation before sending graphs (#2716) (#2733) Convert resource key toples to a string representation before they are submitted to the scheduler. The commit is intended to fix #2716. The test case persists the result of a tiny DataFrame operation and checks the resource restrictions. --- distributed/client.py | 1 + distributed/tests/test_resources.py | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/distributed/client.py b/distributed/client.py index d924b608c6..22d89cda4e 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -2351,6 +2351,7 @@ def _graph_to_futures( resources = self._expand_resources( resources, all_keys=itertools.chain(dsk, keys) ) + resources = {tokey(k): v for k, v in resources.items()} if retries: retries = self._expand_retries( diff --git a/distributed/tests/test_resources.py b/distributed/tests/test_resources.py index d7102ef530..480532d912 100644 --- a/distributed/tests/test_resources.py +++ b/distributed/tests/test_resources.py @@ -202,6 +202,24 @@ def test_persist_tuple(c, s, a, b): assert not b.data +@gen_cluster(client=True) +def test_resources_str(c, s, a, b): + pd = pytest.importorskip("pandas") + dd = pytest.importorskip("dask.dataframe") + + yield a.set_resources(MyRes=1) + + x = dd.from_pandas(pd.DataFrame({"A": [1, 2], "B": [3, 4]}), npartitions=1) + y = x.apply(lambda row: row.sum(), axis=1, meta=(None, "int64")) + yy = y.persist(resources={"MyRes": 1}) + yield wait(yy) + + ts_first = s.tasks[tokey(y.__dask_keys__()[0])] + assert ts_first.resource_restrictions == {"MyRes": 1} + ts_last = s.tasks[tokey(y.__dask_keys__()[-1])] + assert ts_last.resource_restrictions == {"MyRes": 1} + + @gen_cluster( client=True, ncores=[ From bcb765de543a0f15ddce5d1a2e86b7f4bdefde4f Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 3 Jun 2019 17:26:06 -0700 Subject: [PATCH 34/43] Add async context managers to scheduler/worker classes (#2745) --- distributed/node.py | 7 +++++++ distributed/tests/test_scheduler.py | 10 ++++++++++ 2 files changed, 17 insertions(+) diff --git a/distributed/node.py b/distributed/node.py index ff95a62187..4f0b9813a8 100644 --- a/distributed/node.py +++ b/distributed/node.py @@ -130,3 +130,10 @@ def stop_services(self): @property def service_ports(self): return {k: v.port for k, v in self.services.items()} + + async def __aenter__(self): + await self + return self + + async def __aexit__(self, typ, value, traceback): + await self.close() diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 6df271ae34..a0cbfabfa2 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -1572,3 +1572,13 @@ def test_dashboard_address(): s = yield Scheduler(dashboard_address="127.0.0.1", port=0) assert s.services["dashboard"].port yield s.close() + + +@pytest.mark.asyncio +async def test_async_context_manager(): + async with Scheduler(port=0) as s: + assert s.status == "running" + async with Worker(s.address) as w: + assert w.status == "running" + assert s.workers + assert not s.workers From 2a1e089a9dc541a0a19c0fca575c42c719275bd9 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Wed, 5 Jun 2019 10:09:19 -0700 Subject: [PATCH 35/43] Worker dashboard fixes (#2747) * bokeh -> dashboard in template * Add doc to ProfileTimePlot * Add test for bokeh worker routes * Remove info route from worker To do this we ... 1. Remove the baked in "info" link in the base template 2. Add that to the scheduler's list of links 3. Add a redirect from "info" to the actual page 4. Create a generic redirect route 5. Move that and the RequestHandler to utils to avoid code duplication between scheduler and worker Fixes https://github.com/dask/distributed/issues/2722 * Add worker name Fixes https://github.com/dask/dask/issues/4878 --- distributed/dashboard/scheduler.py | 19 +++++++----- distributed/dashboard/scheduler_html.py | 15 ++-------- distributed/dashboard/templates/base.html | 3 -- .../dashboard/templates/worker-table.html | 6 ++-- distributed/dashboard/templates/worker.html | 6 ++-- .../dashboard/tests/test_scheduler_bokeh.py | 6 ++-- .../dashboard/tests/test_worker_bokeh.py | 29 +++++++++++++++++++ distributed/dashboard/utils.py | 20 +++++++++++++ distributed/dashboard/worker.py | 2 +- distributed/dashboard/worker_html.py | 22 ++------------ 10 files changed, 76 insertions(+), 52 deletions(-) diff --git a/distributed/dashboard/scheduler.py b/distributed/dashboard/scheduler.py index 6476d3aa6e..86f56e9eda 100644 --- a/distributed/dashboard/scheduler.py +++ b/distributed/dashboard/scheduler.py @@ -77,7 +77,7 @@ ) template_variables = { - "pages": ["status", "workers", "tasks", "system", "profile", "graph"] + "pages": ["status", "workers", "tasks", "system", "profile", "graph", "info"] } BOKEH_THEME = Theme(os.path.join(os.path.dirname(__file__), "theme.yaml")) @@ -449,7 +449,7 @@ def update(self): "nbytes_text": nbytes_text, "dashboard_host": dashboard_host, "dashboard_port": dashboard_port, - "worker": [ws.address for ws in workers], + "address": [ws.address for ws in workers], "y": y, } @@ -1177,7 +1177,8 @@ class WorkerTable(DashboardComponent): def __init__(self, scheduler, width=800, **kwargs): self.scheduler = scheduler self.names = [ - "worker", + "name", + "address", "ncores", "cpu", "memory", @@ -1195,7 +1196,8 @@ def __init__(self, scheduler, width=800, **kwargs): ) table_names = [ - "worker", + "name", + "address", "ncores", "cpu", "memory", @@ -1242,7 +1244,7 @@ def __init__(self, scheduler, width=800, **kwargs): if name in formatters: table.columns[table_names.index(name)].formatter = formatters[name] - extra_names = ["worker"] + self.extra_names + extra_names = ["name", "address"] + self.extra_names extra_columns = { name: TableColumn(field=name, title=name.replace("_percent", "%")) for name in extra_names @@ -1330,10 +1332,13 @@ def __init__(self, scheduler, width=800, **kwargs): @without_property_validation def update(self): data = {name: [] for name in self.names + self.extra_names} - for addr, ws in sorted(self.scheduler.workers.items()): + for i, (addr, ws) in enumerate( + sorted(self.scheduler.workers.items(), key=lambda kv: kv[1].name) + ): for name in self.names + self.extra_names: data[name].append(ws.metrics.get(name, None)) - data["worker"][-1] = ws.address + data["name"][-1] = ws.name if ws.name is not None else i + data["address"][-1] = ws.address if ws.memory_limit: data["memory_percent"][-1] = ws.metrics["memory"] / ws.memory_limit else: diff --git a/distributed/dashboard/scheduler_html.py b/distributed/dashboard/scheduler_html.py index 5f481f783b..9f2bcd3cbb 100644 --- a/distributed/dashboard/scheduler_html.py +++ b/distributed/dashboard/scheduler_html.py @@ -1,30 +1,18 @@ from datetime import datetime -import os import toolz from tornado import escape from tornado import gen -from tornado import web from ..utils import log_errors, format_bytes, format_time from .proxy import GlobalProxyHandler - -dirname = os.path.dirname(__file__) +from .utils import RequestHandler, redirect ns = { func.__name__: func for func in [format_bytes, format_time, datetime.fromtimestamp] } -class RequestHandler(web.RequestHandler): - def initialize(self, server=None, extra=None): - self.server = server - self.extra = extra or {} - - def get_template_path(self): - return os.path.join(dirname, "templates") - - class Workers(RequestHandler): def get(self): with log_errors(): @@ -238,6 +226,7 @@ def get(self): routes = [ + (r"info", redirect("info/main/workers.html")), (r"info/main/workers.html", Workers), (r"info/worker/(.*).html", Worker), (r"info/task/(.*).html", Task), diff --git a/distributed/dashboard/templates/base.html b/distributed/dashboard/templates/base.html index da15df28b6..83f5e8527c 100644 --- a/distributed/dashboard/templates/base.html +++ b/distributed/dashboard/templates/base.html @@ -29,9 +29,6 @@ {{ page|title }}
  • {% endfor %} -
  • - Info -
Worker Cores {{ len(ws.processing) }} {{ len(ws.has_what) }} bokeh bokeh
+ @@ -13,14 +14,15 @@ {% for ws in worker_list %} + - {% if 'bokeh' in ws.services %} - + {% if 'dashboard' in ws.services %} + {% else %} {% end %} diff --git a/distributed/dashboard/templates/worker.html b/distributed/dashboard/templates/worker.html index 8b26d86e95..9c7608cb8c 100644 --- a/distributed/dashboard/templates/worker.html +++ b/distributed/dashboard/templates/worker.html @@ -1,8 +1,8 @@ {% extends main.html %} {% block content %} -

Worker: {{Worker}}

- {% set ws = workers[Worker] %} - {% set worker_list = [ws] %} +{% set ws = workers[Worker] %} +{% set worker_list = [ws] %} +

Worker: {{ ws.address }}

{% include "worker-table.html" %}
diff --git a/distributed/dashboard/tests/test_scheduler_bokeh.py b/distributed/dashboard/tests/test_scheduler_bokeh.py index f8a813514b..692a29439c 100644 --- a/distributed/dashboard/tests/test_scheduler_bokeh.py +++ b/distributed/dashboard/tests/test_scheduler_bokeh.py @@ -354,7 +354,7 @@ def metric_address(worker): assert all(data.values()) assert all(len(v) == 2 for v in data.values()) - my_index = data["worker"].index(a.address), data["worker"].index(b.address) + my_index = data["address"].index(a.address), data["address"].index(b.address) assert [data["metric_port"][i] for i in my_index] == [a.port, b.port] assert [data["metric_address"][i] for i in my_index] == [a.address, b.address] @@ -379,7 +379,7 @@ def metric_port(worker): assert "metric_b" in data assert all(data.values()) assert all(len(v) == 2 for v in data.values()) - my_index = data["worker"].index(a.address), data["worker"].index(b.address) + my_index = data["address"].index(a.address), data["address"].index(b.address) assert [data["metric_a"][i] for i in my_index] == [a.port, None] assert [data["metric_b"][i] for i in my_index] == [None, b.port] @@ -399,7 +399,7 @@ def metric_port(worker): assert "metric_a" in data assert all(data.values()) assert all(len(v) == 2 for v in data.values()) - my_index = data["worker"].index(a.address), data["worker"].index(b.address) + my_index = data["address"].index(a.address), data["address"].index(b.address) assert [data["metric_a"][i] for i in my_index] == [a.port, None] diff --git a/distributed/dashboard/tests/test_worker_bokeh.py b/distributed/dashboard/tests/test_worker_bokeh.py index 11699d9ac8..ef977127d2 100644 --- a/distributed/dashboard/tests/test_worker_bokeh.py +++ b/distributed/dashboard/tests/test_worker_bokeh.py @@ -1,6 +1,7 @@ from __future__ import print_function, division, absolute_import from operator import add, sub +import re from time import sleep import pytest @@ -14,6 +15,7 @@ from distributed.client import wait from distributed.metrics import time from distributed.utils_test import gen_cluster, inc, dec +from distributed.dashboard.scheduler import BokehScheduler from distributed.dashboard.worker import ( BokehWorker, StateTable, @@ -26,6 +28,33 @@ ) +@gen_cluster( + client=True, + worker_kwargs={"services": {("dashboard", 0): BokehWorker}}, + scheduler_kwargs={"services": {("dashboard", 0): BokehScheduler}}, +) +def test_routes(c, s, a, b): + assert isinstance(a.services["dashboard"], BokehWorker) + assert isinstance(b.services["dashboard"], BokehWorker) + port = a.services["dashboard"].port + + future = c.submit(sleep, 1) + yield gen.sleep(0.1) + + http_client = AsyncHTTPClient() + for suffix in ["status", "counters", "system", "profile", "profile-server"]: + response = yield http_client.fetch("http://localhost:%d/%s" % (port, suffix)) + body = response.body.decode() + assert "bokeh" in body.lower() + assert not re.search("href=./", body) # no absolute links + + response = yield http_client.fetch( + "http://localhost:%d/info/main/workers.html" % s.services["dashboard"].port + ) + + assert str(port) in response.body.decode() + + @pytest.mark.skipif( sys.version_info[0] == 2, reason="https://github.com/bokeh/bokeh/issues/5494" ) diff --git a/distributed/dashboard/utils.py b/distributed/dashboard/utils.py index 516ca5bfb8..a9b31345ca 100644 --- a/distributed/dashboard/utils.py +++ b/distributed/dashboard/utils.py @@ -1,13 +1,16 @@ from __future__ import print_function, division, absolute_import from distutils.version import LooseVersion +import os import bokeh +from tornado import web from toolz import partition from ..compatibility import PY2 BOKEH_VERSION = LooseVersion(bokeh.__version__) +dirname = os.path.dirname(__file__) if BOKEH_VERSION >= "1.0.0" and not PY2: @@ -32,3 +35,20 @@ def parse_args(args): def transpose(lod): keys = list(lod[0].keys()) return {k: [d[k] for d in lod] for k in keys} + + +class RequestHandler(web.RequestHandler): + def initialize(self, server=None, extra=None): + self.server = server + self.extra = extra or {} + + def get_template_path(self): + return os.path.join(dirname, "templates") + + +def redirect(path): + class Redirect(RequestHandler): + def get(self): + self.redirect(path) + + return Redirect diff --git a/distributed/dashboard/worker.py b/distributed/dashboard/worker.py index ed7b68b76b..aa85afc419 100644 --- a/distributed/dashboard/worker.py +++ b/distributed/dashboard/worker.py @@ -735,7 +735,7 @@ def counters_doc(server, extra, doc): def profile_doc(server, extra, doc): with log_errors(): doc.title = "Dask Worker Profile" - profile = ProfileTimePlot(server, sizing_mode="scale_width") + profile = ProfileTimePlot(server, sizing_mode="scale_width", doc=doc) profile.trigger_update() doc.add_root(profile.root) diff --git a/distributed/dashboard/worker_html.py b/distributed/dashboard/worker_html.py index c818c8fb1e..450cce56c8 100644 --- a/distributed/dashboard/worker_html.py +++ b/distributed/dashboard/worker_html.py @@ -1,17 +1,4 @@ -import os - -from tornado import web - -dirname = os.path.dirname(__file__) - - -class RequestHandler(web.RequestHandler): - def initialize(self, server=None, extra=None): - self.server = server - self.extra = extra or {} - - def get_template_path(self): - return os.path.join(dirname, "templates") +from .utils import RequestHandler, redirect class _PrometheusCollector(object): @@ -67,15 +54,10 @@ def get(self): self.set_header("Content-Type", "text/plain") -class OldRoute(RequestHandler): - def get(self): - self.redirect("/status") - - routes = [ (r"metrics", PrometheusHandler), (r"health", HealthHandler), - (r"main", OldRoute), + (r"main", redirect("/status")), ] From e846991d93054a29e28528224c63db69972ffc9c Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Wed, 5 Jun 2019 14:26:50 -0700 Subject: [PATCH 36/43] Add SpecCluster.new_worker_spec method (#2751) * Add type name to LocalCluster.__repr__ * Add SpecCluster.new_worker_spec method This is helpful for subclassing --- distributed/deploy/local.py | 10 ++------- distributed/deploy/spec.py | 22 ++++++++++++++++--- distributed/deploy/tests/test_spec_cluster.py | 18 ++++++++++++++- 3 files changed, 38 insertions(+), 12 deletions(-) diff --git a/distributed/deploy/local.py b/distributed/deploy/local.py index 95f178c7c2..a56cce8c2b 100644 --- a/distributed/deploy/local.py +++ b/distributed/deploy/local.py @@ -197,14 +197,8 @@ def __init__( ) def __repr__(self): - return "LocalCluster(%r, workers=%d, ncores=%d)" % ( - self.scheduler_address, - len(self.workers), - sum(w.ncores for w in self.workers.values()), - ) - - def __repr__(self): - return "LocalCluster(%r, workers=%d, ncores=%d)" % ( + return "%s(%r, workers=%d, ncores=%d)" % ( + type(self).__name__, self.scheduler_address, len(self.workers), sum(w.ncores for w in self.workers.values()), diff --git a/distributed/deploy/spec.py b/distributed/deploy/spec.py index d5a954effc..2558a5df26 100644 --- a/distributed/deploy/spec.py +++ b/distributed/deploy/spec.py @@ -275,12 +275,28 @@ def scale(self, n): self.worker_spec.popitem() while len(self.worker_spec) < n: - while self._i in self.worker_spec: - self._i += 1 - self.worker_spec[self._i] = self.new_spec + k, spec = self.new_worker_spec() + self.worker_spec[k] = spec self.loop.add_callback(self._correct_state) + def new_worker_spec(self): + """ Return name and spec for the next worker + + Returns + ------- + name: identifier for worker + spec: dict + + See Also + -------- + scale + """ + while self._i in self.worker_spec: + self._i += 1 + + return self._i, self.new_spec + async def scale_down(self, workers): workers = set(workers) diff --git a/distributed/deploy/tests/test_spec_cluster.py b/distributed/deploy/tests/test_spec_cluster.py index ac5706afe1..eb733f2e68 100644 --- a/distributed/deploy/tests/test_spec_cluster.py +++ b/distributed/deploy/tests/test_spec_cluster.py @@ -76,7 +76,9 @@ def test_spec_sync(loop): def test_loop_started(): - cluster = SpecCluster(worker_spec) + cluster = SpecCluster( + worker_spec, scheduler={"cls": Scheduler, "options": {"port": 0}} + ) @pytest.mark.asyncio @@ -110,6 +112,7 @@ async def test_broken_worker(): async with SpecCluster( asynchronous=True, workers={"good": {"cls": Worker}, "bad": {"cls": BrokenWorker}}, + scheduler={"cls": Scheduler, "options": {"port": 0}}, ) as cluster: pass @@ -124,3 +127,16 @@ def test_spec_close_clusters(loop): assert cluster in SpecCluster._instances close_clusters() assert cluster.status == "closed" + + +@pytest.mark.asyncio +async def test_new_worker_spec(): + class MyCluster(SpecCluster): + def new_worker_spec(self): + i = len(self.worker_spec) + return i, {"cls": Worker, "options": {"ncores": i + 1}} + + async with MyCluster(asynchronous=True, scheduler=scheduler) as cluster: + cluster.scale(3) + for i in range(3): + assert cluster.worker_spec[i]["options"]["ncores"] == i + 1 From 0696a1f6456b8010b19ac47b14cd2dca0d859246 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Thu, 6 Jun 2019 11:15:33 -0700 Subject: [PATCH 37/43] Move some of the adaptive logic into the scheduler (#2735) * Move some of the adaptive logic into the scheduler * don't close closed clusters * require pytest >= 4 in CI * use worker_spec if it exists * Don't scale a closed cluster * handle intermittent failures --- continuous_integration/travis/install.sh | 2 +- distributed/deploy/adaptive.py | 117 +++------------------- distributed/deploy/spec.py | 7 +- distributed/deploy/tests/test_adaptive.py | 35 +++---- distributed/scheduler.py | 55 ++++++++++ distributed/tests/test_diskutils.py | 2 + distributed/tests/test_scheduler.py | 25 +++++ distributed/utils.py | 12 +-- 8 files changed, 121 insertions(+), 134 deletions(-) diff --git a/continuous_integration/travis/install.sh b/continuous_integration/travis/install.sh index cb2dbdf5c8..2ab9724db2 100644 --- a/continuous_integration/travis/install.sh +++ b/continuous_integration/travis/install.sh @@ -44,7 +44,7 @@ conda install -q \ paramiko \ prometheus_client \ psutil \ - pytest \ + pytest>=4 \ pytest-timeout \ python=$PYTHON \ requests \ diff --git a/distributed/deploy/adaptive.py b/distributed/deploy/adaptive.py index 793e80d984..401acc3dc1 100644 --- a/distributed/deploy/adaptive.py +++ b/distributed/deploy/adaptive.py @@ -4,7 +4,6 @@ import logging import math -import toolz from tornado import gen from ..metrics import time @@ -128,104 +127,6 @@ def stop(self): self._adapt_callback = None del self._adapt_callback - def needs_cpu(self): - """ - Check if the cluster is CPU constrained (too many tasks per core) - - Notes - ----- - Returns ``True`` if the occupancy per core is some factor larger - than ``startup_cost`` and the number of tasks exceeds the number of - cores - """ - total_occupancy = self.scheduler.total_occupancy - total_cores = self.scheduler.total_ncores - - if total_occupancy / (total_cores + 1e-9) > self.startup_cost * 2: - logger.info( - "CPU limit exceeded [%d occupancy / %d cores]", - total_occupancy, - total_cores, - ) - - tasks_processing = 0 - - for w in self.scheduler.workers.values(): - tasks_processing += len(w.processing) - - if tasks_processing > total_cores: - logger.info( - "pending tasks exceed number of cores " "[%d tasks / %d cores]", - tasks_processing, - total_cores, - ) - - return True - - return False - - def needs_memory(self): - """ - Check if the cluster is RAM constrained - - Notes - ----- - Returns ``True`` if the required bytes in distributed memory is some - factor larger than the actual distributed memory available. - """ - limit_bytes = { - addr: ws.memory_limit for addr, ws in self.scheduler.workers.items() - } - worker_bytes = [ws.nbytes for ws in self.scheduler.workers.values()] - - limit = sum(limit_bytes.values()) - total = sum(worker_bytes) - if total > 0.6 * limit: - logger.info("Ram limit exceeded [%d/%d]", limit, total) - return True - else: - return False - - def should_scale_up(self): - """ - Determine whether additional workers should be added to the cluster - - Returns - ------- - scale_up : bool - - Notes - ---- - Additional workers are added whenever - - 1. There are unrunnable tasks and no workers - 2. The cluster is CPU constrained - 3. The cluster is RAM constrained - 4. There are fewer workers than our minimum - - See Also - -------- - needs_cpu - needs_memory - """ - with log_errors(): - if len(self.scheduler.workers) < self.minimum: - return True - - if self.maximum is not None and len(self.scheduler.workers) >= self.maximum: - return False - - if self.scheduler.unrunnable and not self.scheduler.workers: - return True - - needs_cpu = self.needs_cpu() - needs_memory = self.needs_memory() - - if needs_cpu or needs_memory: - return True - - return False - def workers_to_close(self, **kwargs): """ Determine which, if any, workers should potentially be removed from @@ -305,9 +206,17 @@ def get_scale_up_kwargs(self): return {"n": instances} def recommendations(self, comm=None): - should_scale_up = self.should_scale_up() + n = self.scheduler.adaptive_target(target_duration=self.target_duration) + if self.maximum is not None: + n = min(self.maximum, n) + if self.minimum is not None: + n = max(self.minimum, n) workers = set(self.workers_to_close(key=self.worker_key, minimum=self.minimum)) - if should_scale_up and workers: + try: + current = len(self.cluster.worker_spec) + except AttributeError: + current = len(self.cluster.workers) + if n > current and workers: logger.info("Attempting to scale up and scale down simultaneously.") self.close_counts.clear() return { @@ -315,9 +224,9 @@ def recommendations(self, comm=None): "msg": "Trying to scale up and down simultaneously", } - elif should_scale_up: + elif n > current: self.close_counts.clear() - return toolz.merge({"status": "up"}, self.get_scale_up_kwargs()) + return {"status": "up", "n": n} elif workers: d = {} @@ -352,7 +261,7 @@ def _adapt(self): return status = recommendations.pop("status") if status == "up": - f = self.cluster.scale_up(**recommendations) + f = self.cluster.scale(**recommendations) self.log.append((time(), "up", recommendations)) if hasattr(f, "__await__"): yield f diff --git a/distributed/deploy/spec.py b/distributed/deploy/spec.py index 2558a5df26..85728a057e 100644 --- a/distributed/deploy/spec.py +++ b/distributed/deploy/spec.py @@ -274,6 +274,10 @@ def scale(self, n): while len(self.worker_spec) > n: self.worker_spec.popitem() + if self.status in ("closing", "closed"): + self.loop.add_callback(self._correct_state) + return + while len(self.worker_spec) < n: k, spec = self.new_worker_spec() self.worker_spec[k] = spec @@ -321,4 +325,5 @@ def __repr__(self): def close_clusters(): for cluster in list(SpecCluster._instances): with ignoring(gen.TimeoutError): - cluster.close(timeout=10) + if cluster.status != "closed": + cluster.close(timeout=10) diff --git a/distributed/deploy/tests/test_adaptive.py b/distributed/deploy/tests/test_adaptive.py index 8915c72135..cc860636e5 100644 --- a/distributed/deploy/tests/test_adaptive.py +++ b/distributed/deploy/tests/test_adaptive.py @@ -2,13 +2,12 @@ from time import sleep -import pytest from toolz import frequencies, pluck from tornado import gen from tornado.ioloop import IOLoop from distributed import Client, wait, Adaptive, LocalCluster, SpecCluster, Worker -from distributed.utils_test import gen_cluster, gen_test, slowinc, inc, clean +from distributed.utils_test import gen_cluster, gen_test, slowinc, clean from distributed.utils_test import loop, nodebug # noqa: F401 from distributed.metrics import time @@ -116,11 +115,10 @@ def test_adaptive_local_cluster_multi_workers(): yield gen.sleep(0.01) assert time() < start + 15, alc.log - # assert not cluster.workers - assert not cluster.scheduler.workers - yield gen.sleep(0.2) - # assert not cluster.workers - assert not cluster.scheduler.workers + # no workers for a while + for i in range(10): + assert not cluster.scheduler.workers + yield gen.sleep(0.05) futures = c.map(slowinc, range(100), delay=0.01) yield c.gather(futures) @@ -152,6 +150,10 @@ def scale_up(self, n, **kwargs): def scale_down(self, workers): assert False + @property + def workers(self): + return s.workers + assert len(s.workers) == 10 # Assert that adaptive cycle does not reduce cluster below minimum size @@ -163,8 +165,7 @@ def scale_down(self, workers): assert len(s.workers) == 2 -@pytest.mark.xfail(reason="need to rework adaptive") -@gen_test(timeout=30) +@gen_test() def test_min_max(): cluster = yield LocalCluster( 0, @@ -242,7 +243,9 @@ def test_avoid_churn(): yield client.submit(slowinc, i, delay=0.040) yield gen.sleep(0.040) - assert frequencies(pluck(1, adapt.log)) == {"up": 1} + from toolz.curried import pipe, unique, pluck, frequencies + + assert pipe(adapt.log, unique(key=str), pluck(1), frequencies) == {"up": 1} finally: yield client.close() yield cluster.close() @@ -435,15 +438,3 @@ def key(ws): assert names == {"a-1", "a-2"} or names == {"b-1", "b-2"} finally: yield cluster.close() - - -@gen_cluster(client=True, ncores=[]) -def test_without_cluster(c, s): - adapt = Adaptive(scheduler=s) - - future = c.submit(inc, 1) - while not s.tasks: - yield gen.sleep(0.01) - - response = yield c.scheduler.adaptive_recommendations() - assert response["status"] == "up" diff --git a/distributed/scheduler.py b/distributed/scheduler.py index ca3c1241ea..3cf8de4930 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -6,6 +6,7 @@ import itertools import json import logging +import math from numbers import Number import operator import os @@ -1063,6 +1064,7 @@ def __init__( "get_task_status": self.get_task_status, "get_task_stream": self.get_task_stream, "register_worker_plugin": self.register_worker_plugin, + "adaptive_target": self.adaptive_target, } self._transitions = { @@ -4740,6 +4742,59 @@ def check_idle(self): if close: self.loop.add_callback(self.close) + def adaptive_target(self, target_duration="5s"): + """ Desired number of workers based on the current workload + + This looks at the current running tasks and memory use, and returns a + number of desired workers. This is often used by adaptive scheduling. + + Parameters + ---------- + target_duration: str + A desired duration of time for computations to take. This affects + how rapidly the scheduler will ask to scale. + + See Also + -------- + distributed.deploy.Adaptive + """ + target_duration = parse_timedelta(target_duration) + + # CPU + cpu = math.ceil( + self.total_occupancy / target_duration + ) # TODO: threads per worker + + # Avoid a few long tasks from asking for many cores + tasks_processing = 0 + for ws in self.workers.values(): + tasks_processing += len(ws.processing) + + if tasks_processing > cpu: + break + else: + cpu = min(tasks_processing, cpu) + + if self.unrunnable and not self.workers: + cpu = max(1, cpu) + + # Memory + limit_bytes = {addr: ws.memory_limit for addr, ws in self.workers.items()} + worker_bytes = [ws.nbytes for ws in self.workers.values()] + limit = sum(limit_bytes.values()) + total = sum(worker_bytes) + if total > 0.6 * limit: + memory = 2 * len(self.workers) + else: + memory = 0 + + target = max(memory, cpu) + if target >= len(self.workers): + return target + else: # Scale down? + to_close = self.workers_to_close() + return len(self.workers) - len(to_close) + def decide_worker(ts, all_workers, valid_workers, objective): """ diff --git a/distributed/tests/test_diskutils.py b/distributed/tests/test_diskutils.py index 1bededf84a..a6dcf3497a 100644 --- a/distributed/tests/test_diskutils.py +++ b/distributed/tests/test_diskutils.py @@ -276,6 +276,8 @@ def _test_workspace_concurrency(tmpdir, timeout, max_procs): def test_workspace_concurrency(tmpdir): if WINDOWS: raise pytest.xfail.Exception("TODO: unknown failure on windows") + if sys.version_info < (3, 6): + raise pytest.xfail.Exception("TODO: unknown failure on Python 3.5") _test_workspace_concurrency(tmpdir, 2.0, 6) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index a0cbfabfa2..1c321a0290 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -1574,6 +1574,31 @@ def test_dashboard_address(): yield s.close() +@gen_cluster(client=True) +async def test_adaptive_target(c, s, a, b): + assert s.adaptive_target() == 0 + x = c.submit(inc, 1) + await x + assert s.adaptive_target() == 1 + + # Long task + s.task_duration["slowinc"] = 10 + x = c.submit(slowinc, 1, delay=0.5) + while x.key not in s.tasks: + await gen.sleep(0.01) + assert s.adaptive_target(target_duration=".1s") == 1 # still one + + s.task_duration["slowinc"] = 10 + L = c.map(slowinc, range(100), delay=0.5) + while len(s.tasks) < 100: + await gen.sleep(0.01) + assert 10 < s.adaptive_target(target_duration=".1s") <= 100 + del x, L + while s.tasks: + await gen.sleep(0.01) + assert s.adaptive_target(target_duration=".1s") == 0 + + @pytest.mark.asyncio async def test_async_context_manager(): async with Scheduler(port=0) as s: diff --git a/distributed/utils.py b/distributed/utils.py index 55508a4c57..e8de0bc510 100644 --- a/distributed/utils.py +++ b/distributed/utils.py @@ -950,13 +950,13 @@ def tmpfile(extension=""): yield filename if os.path.exists(filename): - if os.path.isdir(filename): - shutil.rmtree(filename) - else: - try: + try: + if os.path.isdir(filename): + shutil.rmtree(filename) + else: os.remove(filename) - except OSError: # sometimes we can't remove a generated temp file - pass + except OSError: # sometimes we can't remove a generated temp file + pass def ensure_bytes(s): From 587be8d48536f52453594eebd1a23becf864ccf9 Mon Sep 17 00:00:00 2001 From: Tom Augspurger Date: Thu, 6 Jun 2019 16:42:32 -0500 Subject: [PATCH 38/43] Add nanny logs (#2744) --- distributed/client.py | 20 ++++++++++++++------ distributed/nanny.py | 2 ++ distributed/node.py | 24 +++++++++++++++++++++++- distributed/scheduler.py | 28 +++++----------------------- distributed/tests/test_client.py | 16 ++++++++++++++-- distributed/worker.py | 24 ++---------------------- docs/source/api.rst | 2 +- 7 files changed, 61 insertions(+), 55 deletions(-) diff --git a/distributed/client.py b/distributed/client.py index 22d89cda4e..ac098d7987 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -2270,6 +2270,10 @@ def run(self, function, *args, **kwargs): wait: boolean (optional) If the function is asynchronous whether or not to wait until that function finishes. + nanny : bool, defualt False + Whether to run ``function`` on the nanny. By default, the function + is run on the worker process. If specified, the addresses in + ``workers`` should still be the worker addresses, not the nanny addresses. Examples -------- @@ -3354,7 +3358,7 @@ def get_scheduler_logs(self, n=None): Parameters ---------- - n: int + n : int Number of logs to retrive. Maxes out at 10000 by default, confiruable in config.yaml::log-length @@ -3364,23 +3368,27 @@ def get_scheduler_logs(self, n=None): """ return self.sync(self.scheduler.logs, n=n) - def get_worker_logs(self, n=None, workers=None): + def get_worker_logs(self, n=None, workers=None, nanny=False): """ Get logs from workers Parameters ---------- - n: int + n : int Number of logs to retrive. Maxes out at 10000 by default, confiruable in config.yaml::log-length - workers: iterable - List of worker addresses to retrive. Gets all workers by default. + workers : iterable + List of worker addresses to retrieve. Gets all workers by default. + nanny : bool, default False + Whether to get the logs from the workers (False) or the nannies (True). If + specified, the addresses in `workers` should still be the worker addresses, + not the nanny addresses. Returns ------- Dictionary mapping worker address to logs. Logs are returned in reversed order (newest first) """ - return self.sync(self.scheduler.worker_logs, n=n, workers=workers) + return self.sync(self.scheduler.worker_logs, n=n, workers=workers, nanny=nanny) def retire_workers(self, workers=None, close_workers=True, **kwargs): """ Retire certain workers on the scheduler diff --git a/distributed/nanny.py b/distributed/nanny.py index 59a8083e83..9cf444fc7c 100644 --- a/distributed/nanny.py +++ b/distributed/nanny.py @@ -79,6 +79,7 @@ def __init__( protocol=None, **worker_kwargs ): + self._setup_logging(logger) self.loop = loop or IOLoop.current() self.security = security or Security() assert isinstance(self.security, Security) @@ -130,6 +131,7 @@ def __init__( "kill": self.kill, "restart": self.restart, # cannot call it 'close' on the rpc side for naming conflict + "get_logs": self.get_logs, "terminate": self.close, "close_gracefully": self.close_gracefully, "run": self.run, diff --git a/distributed/node.py b/distributed/node.py index 4f0b9813a8..8bd81ffe5a 100644 --- a/distributed/node.py +++ b/distributed/node.py @@ -1,12 +1,15 @@ from __future__ import print_function, division, absolute_import import warnings +import logging from tornado.ioloop import IOLoop +import dask -from .compatibility import unicode +from .compatibility import unicode, finalize from .core import Server, ConnectionPool from .versions import get_versions +from .utils import DequeHandler class Node(object): @@ -131,6 +134,25 @@ def stop_services(self): def service_ports(self): return {k: v.port for k, v in self.services.items()} + def _setup_logging(self, logger): + self._deque_handler = DequeHandler( + n=dask.config.get("distributed.admin.log-length") + ) + self._deque_handler.setFormatter( + logging.Formatter(dask.config.get("distributed.admin.log-format")) + ) + logger.addHandler(self._deque_handler) + finalize(self, logger.removeHandler, self._deque_handler) + + def get_logs(self, comm=None, n=None): + deque_handler = self._deque_handler + if n is None: + L = list(deque_handler.deque) + else: + L = deque_handler.deque + L = [L[-i] for i in range(min(n, len(L)))] + return [(msg.levelname, deque_handler.format(msg)) for msg in L] + async def __aenter__(self): await self return self diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 3cf8de4930..2705971e15 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -53,7 +53,6 @@ key_split, validate_key, no_default, - DequeHandler, parse_timedelta, parse_bytes, PeriodicCallback, @@ -844,7 +843,7 @@ def __init__( dashboard_address=None, **kwargs ): - self._setup_logging() + self._setup_logging(logger) # Attributes self.allowed_failures = allowed_failures @@ -1329,16 +1328,6 @@ def close_worker(self, stream=None, worker=None, safe=None): self.worker_send(worker, {"op": "close", "report": False}) self.remove_worker(address=worker, safe=safe) - def _setup_logging(self): - self._deque_handler = DequeHandler( - n=dask.config.get("distributed.admin.log-length") - ) - self._deque_handler.setFormatter( - logging.Formatter(dask.config.get("distributed.admin.log-format")) - ) - logger.addHandler(self._deque_handler) - finalize(self, logger.removeHandler, self._deque_handler) - ########### # Stimuli # ########### @@ -4627,18 +4616,11 @@ def get_profile_metadata( raise gen.Return({"counts": counts, "keys": keys}) - def get_logs(self, comm=None, n=None): - deque_handler = self._deque_handler - if n is None: - L = list(deque_handler.deque) - else: - L = deque_handler.deque - L = [L[-i] for i in range(min(n, len(L)))] - return [(msg.levelname, deque_handler.format(msg)) for msg in L] - @gen.coroutine - def get_worker_logs(self, comm=None, n=None, workers=None): - results = yield self.broadcast(msg={"op": "get_logs", "n": n}, workers=workers) + def get_worker_logs(self, comm=None, n=None, workers=None, nanny=False): + results = yield self.broadcast( + msg={"op": "get_logs", "n": n}, workers=workers, nanny=nanny + ) raise gen.Return(results) ########### diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index c731ae6e5a..dc45b3025e 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -5111,7 +5111,7 @@ def test_task_metadata(c, s, a, b): assert result == {"a": {"c": {"d": 1}}, "b": 2} -@gen_cluster(client=True) +@gen_cluster(client=True, Worker=Nanny) def test_logs(c, s, a, b): yield wait(c.map(inc, range(5))) logs = yield c.get_scheduler_logs(n=5) @@ -5121,11 +5121,23 @@ def test_logs(c, s, a, b): assert "distributed.scheduler" in msg w_logs = yield c.get_worker_logs(n=5) - assert set(w_logs.keys()) == {a.address, b.address} + assert set(w_logs.keys()) == {a.worker_address, b.worker_address} for log in w_logs.values(): for _, msg in log: assert "distributed.worker" in msg + n_logs = yield c.get_worker_logs(nanny=True) + assert set(n_logs.keys()) == {a.worker_address, b.worker_address} + for log in n_logs.values(): + for _, msg in log: + assert "distributed.nanny" in msg + + n_logs = yield c.get_worker_logs(nanny=True, workers=[a.worker_address]) + assert set(n_logs.keys()) == {a.worker_address} + for log in n_logs.values(): + for _, msg in log: + assert "distributed.nanny" in msg + @gen_cluster(client=True) def test_avoid_delayed_finalize(c, s, a, b): diff --git a/distributed/worker.py b/distributed/worker.py index d0bc735ec6..37dcbc2eca 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -33,7 +33,7 @@ from .comm import get_address_host, get_local_address_for, connect from .comm.utils import offload from .comm.addressing import address_from_user_args -from .compatibility import unicode, get_thread_identity, finalize, MutableMapping +from .compatibility import unicode, get_thread_identity, MutableMapping from .core import error_message, CommClosedError, send_recv, pingpong, coerce_to_address from .diskutils import WorkSpace from .metrics import time @@ -60,7 +60,6 @@ json_load_robust, key_split, format_bytes, - DequeHandler, PeriodicCallback, parse_bytes, parse_timedelta, @@ -412,7 +411,7 @@ def __init__( ) profile_cycle_interval = parse_timedelta(profile_cycle_interval, default="ms") - self._setup_logging() + self._setup_logging(logger) if scheduler_file: cfg = json_load_robust(scheduler_file) @@ -666,16 +665,6 @@ def __repr__(self): ) ) - def _setup_logging(self): - self._deque_handler = DequeHandler( - n=dask.config.get("distributed.admin.log-length") - ) - self._deque_handler.setFormatter( - logging.Formatter(dask.config.get("distributed.admin.log-format")) - ) - logger.addHandler(self._deque_handler) - finalize(self, logger.removeHandler, self._deque_handler) - @property def worker_address(self): """ For API compatibility with Nanny """ @@ -888,15 +877,6 @@ def gather(self, comm=None, who_has=None): self.update_data(data=result, report=False) raise Return({"status": "OK"}) - def get_logs(self, comm=None, n=None): - deque_handler = self._deque_handler - if n is None: - L = list(deque_handler.deque) - else: - L = deque_handler.deque - L = [L[-i] for i in range(min(n, len(L)))] - return [(msg.levelname, deque_handler.format(msg)) for msg in L] - ############# # Lifecycle # ############# diff --git a/docs/source/api.rst b/docs/source/api.rst index e91c4ee6ac..574a70d34b 100644 --- a/docs/source/api.rst +++ b/docs/source/api.rst @@ -19,8 +19,8 @@ API Client.get_executor Client.get_metadata Client.get_scheduler_logs - Client.get_task_stream Client.get_worker_logs + Client.get_task_stream Client.has_what Client.list_datasets Client.map From 5042f579b9b77576da319995cf36d0798875b621 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Fri, 7 Jun 2019 12:11:49 -0700 Subject: [PATCH 39/43] Add stress test for UCX (#2759) This test generated https://github.com/rapidsai/ucx-py/pull/120 --- distributed/comm/tests/test_ucx.py | 31 ++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/distributed/comm/tests/test_ucx.py b/distributed/comm/tests/test_ucx.py index 55a2f4ec82..8a0e8927cf 100644 --- a/distributed/comm/tests/test_ucx.py +++ b/distributed/comm/tests/test_ucx.py @@ -294,3 +294,34 @@ def test_tcp_localcluster(loop): # assert any(w.data == {x.key: 2} for w in c.workers) # assert e.loop is c.loop # print(c.scheduler.workers) + + +@pytest.mark.slow +@pytest.mark.asyncio +async def test_stress(): + from distributed.utils import get_ip_interface + + try: # this check should be removed once UCX + TCP works + get_ip_interface("ib0") + except Exception: + pytest.skip("ib0 interface not found") + + import dask.array as da + from distributed import wait + + chunksize = "10 MB" + + async with LocalCluster( + protocol="ucx", interface="ib0", asynchronous=True + ) as cluster: + async with Client(cluster, asynchronous=True) as client: + rs = da.random.RandomState() + x = rs.random((10000, 10000), chunks=(-1, chunksize)) + x = x.persist() + await wait(x) + + for i in range(10): + x = x.rechunk((chunksize, -1)) + x = x.rechunk((-1, chunksize)) + x = x.persist() + await wait(x) From 756bdd8eb891ee09af6340b7fef4bd883d9fcefb Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Fri, 7 Jun 2019 13:04:21 -0700 Subject: [PATCH 40/43] Remove module state in Prometheus Handlers (#2760) This also fixes an ImportError in prometheus-client=0.7 --- distributed/dashboard/scheduler_html.py | 24 ++++++++++-------------- distributed/dashboard/worker_html.py | 18 +++++++----------- 2 files changed, 17 insertions(+), 25 deletions(-) diff --git a/distributed/dashboard/scheduler_html.py b/distributed/dashboard/scheduler_html.py index 9f2bcd3cbb..8b1da2035e 100644 --- a/distributed/dashboard/scheduler_html.py +++ b/distributed/dashboard/scheduler_html.py @@ -175,17 +175,18 @@ def get(self): class _PrometheusCollector(object): - def __init__(self, server, prometheus_client): + def __init__(self, server): self.server = server - self.prometheus_client = prometheus_client def collect(self): - yield self.prometheus_client.core.GaugeMetricFamily( + from prometheus_client.core import GaugeMetricFamily + + yield GaugeMetricFamily( "dask_scheduler_workers", "Number of workers.", value=len(self.server.workers), ) - yield self.prometheus_client.core.GaugeMetricFamily( + yield GaugeMetricFamily( "dask_scheduler_clients", "Number of clients.", value=len(self.server.clients), @@ -196,26 +197,21 @@ class PrometheusHandler(RequestHandler): _initialized = False def __init__(self, *args, **kwargs): - import prometheus_client # keep out of global namespace - - self.prometheus_client = prometheus_client + import prometheus_client super(PrometheusHandler, self).__init__(*args, **kwargs) - self._init() - - def _init(self): if PrometheusHandler._initialized: return - self.prometheus_client.REGISTRY.register( - _PrometheusCollector(self.server, self.prometheus_client) - ) + prometheus_client.REGISTRY.register(_PrometheusCollector(self.server)) PrometheusHandler._initialized = True def get(self): - self.write(self.prometheus_client.generate_latest()) + import prometheus_client + + self.write(prometheus_client.generate_latest()) self.set_header("Content-Type", "text/plain; version=0.0.4") diff --git a/distributed/dashboard/worker_html.py b/distributed/dashboard/worker_html.py index 450cce56c8..e1ae50f3af 100644 --- a/distributed/dashboard/worker_html.py +++ b/distributed/dashboard/worker_html.py @@ -4,7 +4,6 @@ class _PrometheusCollector(object): def __init__(self, server, prometheus_client): self.server = server - self.prometheus_client = prometheus_client def collect(self): # add your metrics here: @@ -14,7 +13,7 @@ def collect(self): yield None # # 2. yield your metrics - # yield self.prometheus_client.core.GaugeMetricFamily( + # yield prometheus_client.core.GaugeMetricFamily( # 'dask_worker_connections', # 'Number of connections currently open.', # value=???, @@ -25,26 +24,23 @@ class PrometheusHandler(RequestHandler): _initialized = False def __init__(self, *args, **kwargs): - import prometheus_client # keep out of global namespace - - self.prometheus_client = prometheus_client + import prometheus_client super(PrometheusHandler, self).__init__(*args, **kwargs) - self._init() - - def _init(self): if PrometheusHandler._initialized: return - self.prometheus_client.REGISTRY.register( - _PrometheusCollector(self.server, self.prometheus_client) + prometheus_client.REGISTRY.register( + _PrometheusCollector(self.server, prometheus_client) ) PrometheusHandler._initialized = True def get(self): - self.write(self.prometheus_client.generate_latest()) + import prometheus_client + + self.write(prometheus_client.generate_latest()) self.set_header("Content-Type", "text/plain; version=0.0.4") From 309e435cbb383e437bb8af3c571b52fb163a0ac9 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Fri, 7 Jun 2019 13:25:56 -0700 Subject: [PATCH 41/43] Change address -> worker in ColumnDataSource for nbytes plot (#2755) Fixes #2754 --- distributed/dashboard/scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/dashboard/scheduler.py b/distributed/dashboard/scheduler.py index 86f56e9eda..2cb916d0b5 100644 --- a/distributed/dashboard/scheduler.py +++ b/distributed/dashboard/scheduler.py @@ -449,7 +449,7 @@ def update(self): "nbytes_text": nbytes_text, "dashboard_host": dashboard_host, "dashboard_port": dashboard_port, - "address": [ws.address for ws in workers], + "worker": [ws.address for ws in workers], "y": y, } From d378b41a89e33e660257522dd4b86d44e6d15fc5 Mon Sep 17 00:00:00 2001 From: Tom Augspurger Date: Sat, 8 Jun 2019 13:29:29 -0500 Subject: [PATCH 42/43] Delay lookup of allowed failures. (#2761) This allows for setting the config after importing distributed xref https://github.com/dask/dask-examples/pull/75#discussion_r291141404 --- distributed/scheduler.py | 6 +++--- distributed/tests/test_scheduler.py | 14 ++++++++++++++ 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 2705971e15..ae449bcfaf 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -73,8 +73,6 @@ logger = logging.getLogger(__name__) -ALLOWED_FAILURES = dask.config.get("distributed.scheduler.allowed-failures") - LOG_PDB = dask.config.get("distributed.admin.pdb-on-err") DEFAULT_DATA_SIZE = dask.config.get("distributed.scheduler.default-data-size") @@ -829,7 +827,7 @@ def __init__( synchronize_worker_interval="60s", services=None, service_kwargs=None, - allowed_failures=ALLOWED_FAILURES, + allowed_failures=None, extensions=None, validate=False, scheduler_file=None, @@ -846,6 +844,8 @@ def __init__( self._setup_logging(logger) # Attributes + if allowed_failures is None: + allowed_failures = dask.config.get("distributed.scheduler.allowed-failures") self.allowed_failures = allowed_failures self.validate = validate self.status = None diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 1c321a0290..66a8088ace 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -1607,3 +1607,17 @@ async def test_async_context_manager(): assert w.status == "running" assert s.workers assert not s.workers + + +@pytest.mark.asyncio +async def test_allowed_failures_config(): + async with Scheduler(port=0, allowed_failures=10) as s: + assert s.allowed_failures == 10 + + with dask.config.set({"distributed.scheduler.allowed_failures": 100}): + async with Scheduler(port=0) as s: + assert s.allowed_failures == 100 + + with dask.config.set({"distributed.scheduler.allowed_failures": 0}): + async with Scheduler(port=0) as s: + assert s.allowed_failures == 0 From 2ba70b310dbefc5764ee43079ac1bb783a8cec08 Mon Sep 17 00:00:00 2001 From: Tom Augspurger Date: Mon, 10 Jun 2019 16:31:33 -0500 Subject: [PATCH 43/43] Add unknown pytest markers (#2764) --- setup.cfg | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/setup.cfg b/setup.cfg index 434b1fd258..5533437121 100644 --- a/setup.cfg +++ b/setup.cfg @@ -41,6 +41,11 @@ universal=1 [tool:pytest] addopts = -rsx -v --durations=10 minversion = 3.2 +markers = + slow: marks tests as slow (deselect with '-m "not slow"') + avoid_travis: marks tests as flaky on TravisCI. + ipython: mark a test as exercising IPython + # filterwarnings = # error # ignore::UserWarning
Worker Name Cores Memory Memory use
{{ws.address}} {{ ws.name if ws.name is not None else "" }} {{ ws.ncores }} {{ format_bytes(ws.memory_limit) }} {{ format_time(ws.occupancy) }} {{ len(ws.processing) }} {{ len(ws.has_what) }} bokeh dashboard